You can not select more than 25 topics
			Topics must start with a letter or number, can include dashes ('-') and can be up to 35 characters long.
		
		
		
		
		
			
		
			
				
					
					
						
							438 lines
						
					
					
						
							13 KiB
						
					
					
				
			
		
		
		
			
			
			
		
		
	
	
							438 lines
						
					
					
						
							13 KiB
						
					
					
				| syntax = "proto3"; | |
|  | |
| package messaging_pb; | |
|  | |
| import "mq_schema.proto"; | |
| import "filer.proto"; | |
|  | |
| option go_package = "github.com/seaweedfs/seaweedfs/weed/pb/mq_pb"; | |
| option java_package = "seaweedfs.mq"; | |
| option java_outer_classname = "MessageQueueProto"; | |
|  | |
| ////////////////////////////////////////////////// | |
|  | |
| service SeaweedMessaging { | |
|  | |
|     // control plane | |
|     rpc FindBrokerLeader (FindBrokerLeaderRequest) returns (FindBrokerLeaderResponse) { | |
|     } | |
|  | |
|     // control plane for balancer | |
|     rpc PublisherToPubBalancer (stream PublisherToPubBalancerRequest) returns (stream PublisherToPubBalancerResponse) { | |
|     } | |
|     rpc BalanceTopics (BalanceTopicsRequest) returns (BalanceTopicsResponse) { | |
|     } | |
|  | |
|     // control plane for topic partitions | |
|     rpc ListTopics (ListTopicsRequest) returns (ListTopicsResponse) { | |
|     } | |
|     rpc TopicExists (TopicExistsRequest) returns (TopicExistsResponse) { | |
|     } | |
|     rpc ConfigureTopic (ConfigureTopicRequest) returns (ConfigureTopicResponse) { | |
|     } | |
|     rpc LookupTopicBrokers (LookupTopicBrokersRequest) returns (LookupTopicBrokersResponse) { | |
|     } | |
|     rpc GetTopicConfiguration (GetTopicConfigurationRequest) returns (GetTopicConfigurationResponse) { | |
|     } | |
|     rpc GetTopicPublishers (GetTopicPublishersRequest) returns (GetTopicPublishersResponse) { | |
|     } | |
|     rpc GetTopicSubscribers (GetTopicSubscribersRequest) returns (GetTopicSubscribersResponse) { | |
|     } | |
|  | |
|     // invoked by the balancer, running on each broker | |
|     rpc AssignTopicPartitions (AssignTopicPartitionsRequest) returns (AssignTopicPartitionsResponse) { | |
|     } | |
|     rpc ClosePublishers(ClosePublishersRequest) returns (ClosePublishersResponse) { | |
|     } | |
|     rpc CloseSubscribers(CloseSubscribersRequest) returns (CloseSubscribersResponse) { | |
|     } | |
|  | |
|     // subscriber connects to broker balancer, which coordinates with the subscribers | |
|     rpc SubscriberToSubCoordinator (stream SubscriberToSubCoordinatorRequest) returns (stream SubscriberToSubCoordinatorResponse) { | |
|     } | |
|  | |
|     // data plane for each topic partition | |
|     rpc PublishMessage (stream PublishMessageRequest) returns (stream PublishMessageResponse) { | |
|     } | |
|     rpc SubscribeMessage (stream SubscribeMessageRequest) returns (stream SubscribeMessageResponse) { | |
|     } | |
|     // The lead broker asks a follower broker to follow itself | |
|     rpc PublishFollowMe (stream PublishFollowMeRequest) returns (stream PublishFollowMeResponse) { | |
|     } | |
|     rpc SubscribeFollowMe (stream SubscribeFollowMeRequest) returns (SubscribeFollowMeResponse) { | |
|     } | |
|      | |
|     // SQL query support - get unflushed messages from broker's in-memory buffer (streaming) | |
|     rpc GetUnflushedMessages (GetUnflushedMessagesRequest) returns (stream GetUnflushedMessagesResponse) { | |
|     } | |
|  | |
|     // Get comprehensive partition range information (offsets, timestamps, and other fields) | |
|     rpc GetPartitionRangeInfo (GetPartitionRangeInfoRequest) returns (GetPartitionRangeInfoResponse) { | |
|     } | |
|  | |
|     // Removed Kafka Gateway Registration - no longer needed | |
| } | |
|  | |
| ////////////////////////////////////////////////// | |
|  | |
| message FindBrokerLeaderRequest { | |
|     string filer_group = 1; | |
| } | |
|  | |
| message FindBrokerLeaderResponse { | |
|     string broker = 1; | |
| } | |
|  | |
| ////////////////////////////////////////////////// | |
| message BrokerStats { | |
|     int32 cpu_usage_percent = 1; | |
|     map<string, TopicPartitionStats> stats = 2; | |
| } | |
| message TopicPartitionStats { | |
|     schema_pb.Topic topic = 1; | |
|     schema_pb.Partition partition = 2; | |
|     int32 publisher_count = 3; | |
|     int32 subscriber_count = 4; | |
|     string follower = 5; | |
| } | |
|  | |
|  | |
| message PublisherToPubBalancerRequest { | |
|     message InitMessage { | |
|         string broker = 1; | |
|     } | |
|     oneof message { | |
|         InitMessage init = 1; | |
|         BrokerStats stats = 2; | |
|     } | |
| } | |
| message PublisherToPubBalancerResponse { | |
| } | |
|  | |
| message BalanceTopicsRequest { | |
| } | |
| message BalanceTopicsResponse { | |
| } | |
|  | |
| ////////////////////////////////////////////////// | |
| message TopicRetention { | |
|     int64 retention_seconds = 1; // retention duration in seconds | |
|     bool enabled = 2; // whether retention is enabled | |
| } | |
|  | |
| message ConfigureTopicRequest { | |
|     schema_pb.Topic topic = 1; | |
|     int32 partition_count = 2; | |
|     TopicRetention retention = 3; | |
|     schema_pb.RecordType message_record_type = 4;  // Complete flat schema for the message | |
|     repeated string key_columns = 5;              // Names of columns that form the key | |
|     string schema_format = 6;                      // Serialization format: "AVRO", "PROTOBUF", "JSON_SCHEMA", or empty for schemaless | |
| } | |
| message ConfigureTopicResponse { | |
|     repeated BrokerPartitionAssignment broker_partition_assignments = 2; | |
|     TopicRetention retention = 3; | |
|     schema_pb.RecordType message_record_type = 4;  // Complete flat schema for the message | |
|     repeated string key_columns = 5;              // Names of columns that form the key | |
|     string schema_format = 6;                      // Serialization format: "AVRO", "PROTOBUF", "JSON_SCHEMA", or empty for schemaless | |
| } | |
| message ListTopicsRequest { | |
| } | |
| message ListTopicsResponse { | |
|     repeated schema_pb.Topic topics = 1; | |
| } | |
| message TopicExistsRequest { | |
|     schema_pb.Topic topic = 1; | |
| } | |
| message TopicExistsResponse { | |
|     bool exists = 1; | |
| } | |
| message LookupTopicBrokersRequest { | |
|     schema_pb.Topic topic = 1; | |
| } | |
| message LookupTopicBrokersResponse { | |
|     schema_pb.Topic topic = 1; | |
|     repeated BrokerPartitionAssignment broker_partition_assignments = 2; | |
| } | |
| message BrokerPartitionAssignment { | |
|     schema_pb.Partition partition = 1; | |
|     string leader_broker = 2; | |
|     string follower_broker = 3; | |
| } | |
| message GetTopicConfigurationRequest { | |
|     schema_pb.Topic topic = 1; | |
| } | |
| message GetTopicConfigurationResponse { | |
|     schema_pb.Topic topic = 1; | |
|     int32 partition_count = 2; | |
|     repeated BrokerPartitionAssignment broker_partition_assignments = 3; | |
|     int64 created_at_ns = 4; | |
|     int64 last_updated_ns = 5; | |
|     TopicRetention retention = 6; | |
|     schema_pb.RecordType message_record_type = 7;  // Complete flat schema for the message | |
|     repeated string key_columns = 8;              // Names of columns that form the key | |
|     string schema_format = 9;                      // Serialization format: "AVRO", "PROTOBUF", "JSON_SCHEMA", or empty for schemaless | |
| } | |
|  | |
| message GetTopicPublishersRequest { | |
|     schema_pb.Topic topic = 1; | |
| } | |
| message GetTopicPublishersResponse { | |
|     repeated TopicPublisher publishers = 1; | |
| } | |
|  | |
| message GetTopicSubscribersRequest { | |
|     schema_pb.Topic topic = 1; | |
| } | |
| message GetTopicSubscribersResponse { | |
|     repeated TopicSubscriber subscribers = 1; | |
| } | |
|  | |
| message TopicPublisher { | |
|     string publisher_name = 1; | |
|     string client_id = 2; | |
|     schema_pb.Partition partition = 3; | |
|     int64 connect_time_ns = 4; | |
|     int64 last_seen_time_ns = 5; | |
|     string broker = 6; | |
|     bool is_active = 7; | |
|     int64 last_published_offset = 8; | |
|     int64 last_acked_offset = 9; | |
| } | |
|  | |
| message TopicSubscriber { | |
|     string consumer_group = 1; | |
|     string consumer_id = 2; | |
|     string client_id = 3; | |
|     schema_pb.Partition partition = 4; | |
|     int64 connect_time_ns = 5; | |
|     int64 last_seen_time_ns = 6; | |
|     string broker = 7; | |
|     bool is_active = 8; | |
|     int64 current_offset = 9; // last acknowledged offset | |
|     int64 last_received_offset = 10; | |
| } | |
|  | |
| message AssignTopicPartitionsRequest { | |
|     schema_pb.Topic topic = 1; | |
|     repeated BrokerPartitionAssignment broker_partition_assignments = 2; | |
|     bool is_leader = 3; | |
|     bool is_draining = 4; | |
| } | |
| message AssignTopicPartitionsResponse { | |
| } | |
|  | |
| message SubscriberToSubCoordinatorRequest { | |
|     message InitMessage { | |
|         string consumer_group = 1; | |
|         string consumer_group_instance_id = 2; | |
|         schema_pb.Topic topic = 3; | |
|         // The consumer group instance will be assigned at most max_partition_count partitions. | |
|         // If the number of partitions is less than the sum of max_partition_count, | |
|         // the consumer group instance may be assigned partitions less than max_partition_count. | |
|         // Default is 1. | |
|         int32 max_partition_count = 4; | |
|         // If consumer group instance changes, wait for rebalance_seconds before reassigning partitions | |
|         // Exception: if adding a new consumer group instance and sum of max_partition_count equals the number of partitions, | |
|         // the rebalance will happen immediately. | |
|         // Default is 10 seconds. | |
|         int32 rebalance_seconds = 5; | |
|     } | |
|     message AckUnAssignmentMessage { | |
|         schema_pb.Partition partition = 1; | |
|     } | |
|     message AckAssignmentMessage { | |
|         schema_pb.Partition partition = 1; | |
|     } | |
|     oneof message { | |
|         InitMessage init = 1; | |
|         AckAssignmentMessage ack_assignment = 2; | |
|         AckUnAssignmentMessage ack_un_assignment = 3; | |
|     } | |
| } | |
| message SubscriberToSubCoordinatorResponse { | |
|     message Assignment { | |
|         BrokerPartitionAssignment partition_assignment = 1; | |
|     } | |
|     message UnAssignment { | |
|         schema_pb.Partition partition = 1; | |
|     } | |
|     oneof message { | |
|         Assignment assignment = 1; | |
|         UnAssignment un_assignment = 2; | |
|     } | |
| } | |
|  | |
| ////////////////////////////////////////////////// | |
| message ControlMessage { | |
|     bool is_close = 1; | |
|     string publisher_name = 2; | |
| } | |
| message DataMessage { | |
|     bytes key = 1; | |
|     bytes value = 2; | |
|     int64 ts_ns = 3; | |
|     ControlMessage ctrl = 4; | |
| } | |
| message PublishMessageRequest { | |
|     message InitMessage { | |
|         schema_pb.Topic topic = 1; | |
|         schema_pb.Partition partition = 2; | |
|         int32 ack_interval = 3; | |
|         string follower_broker = 4; | |
|         string publisher_name = 5; // for debugging | |
|     } | |
|     oneof message { | |
|         InitMessage init = 1; | |
|         DataMessage data = 2; | |
|     } | |
| } | |
| message PublishMessageResponse { | |
|     int64 ack_ts_ns = 1;  // Acknowledgment timestamp in nanoseconds | |
|     string error = 2; | |
|     bool should_close = 3; | |
|     int32 error_code = 4; // Structured error code for reliable error mapping | |
|     int64 assigned_offset = 5; // The actual offset assigned by SeaweedMQ for this message | |
| } | |
| message PublishFollowMeRequest { | |
|     message InitMessage { | |
|         schema_pb.Topic topic = 1; | |
|         schema_pb.Partition partition = 2; | |
|     } | |
|     message FlushMessage { | |
|         int64 ts_ns = 1; | |
|     } | |
|     message CloseMessage { | |
|     } | |
|     oneof message { | |
|         InitMessage init = 1; | |
|         DataMessage data = 2; | |
|         FlushMessage flush = 3; | |
|         CloseMessage close = 4; | |
|     } | |
| } | |
| message PublishFollowMeResponse { | |
|     int64 ack_ts_ns = 1; | |
| } | |
| message SubscribeMessageRequest { | |
|     message InitMessage { | |
|         string consumer_group = 1; | |
|         string consumer_id = 2; | |
|         string client_id = 3; | |
|         schema_pb.Topic topic = 4; | |
|         schema_pb.PartitionOffset partition_offset = 5; | |
|         schema_pb.OffsetType offset_type = 6; | |
|         string filter = 10; | |
|         string follower_broker = 11; | |
|         int32 sliding_window_size = 12; | |
|     } | |
|     message AckMessage { | |
|         int64 ts_ns = 1;  // Timestamp in nanoseconds for acknowledgment tracking | |
|         bytes key = 2; | |
|     } | |
|     oneof message { | |
|         InitMessage init = 1; | |
|         AckMessage ack = 2; | |
|     } | |
| } | |
| message SubscribeMessageResponse { | |
|     message SubscribeCtrlMessage { | |
|         string error = 1; | |
|         bool is_end_of_stream = 2; | |
|         bool is_end_of_topic = 3; | |
|     } | |
|     oneof message { | |
|         SubscribeCtrlMessage ctrl = 1; | |
|         DataMessage data = 2; | |
|     } | |
| } | |
| message SubscribeFollowMeRequest { | |
|     message InitMessage { | |
|         schema_pb.Topic topic = 1; | |
|         schema_pb.Partition partition = 2; | |
|         string consumer_group = 3; | |
|     } | |
|     message AckMessage { | |
|         int64 ts_ns = 1; | |
|     } | |
|     message CloseMessage { | |
|     } | |
|     oneof message { | |
|         InitMessage init = 1; | |
|         AckMessage ack = 2; | |
|         CloseMessage close = 3; | |
|     } | |
| } | |
| message SubscribeFollowMeResponse { | |
|     int64 ack_ts_ns = 1; | |
| } | |
| message ClosePublishersRequest { | |
|     schema_pb.Topic topic = 1; | |
|     int64 unix_time_ns = 2; | |
| } | |
| message ClosePublishersResponse { | |
| } | |
| message CloseSubscribersRequest { | |
|     schema_pb.Topic topic = 1; | |
|     int64 unix_time_ns = 2; | |
| } | |
| message CloseSubscribersResponse { | |
| } | |
|  | |
| ////////////////////////////////////////////////// | |
| // SQL query support messages | |
|  | |
| message GetUnflushedMessagesRequest { | |
|     schema_pb.Topic topic = 1; | |
|     schema_pb.Partition partition = 2; | |
|     int64 start_buffer_offset = 3; // Filter by buffer offset (messages from buffers >= this offset) | |
| } | |
|  | |
| message GetUnflushedMessagesResponse { | |
|     filer_pb.LogEntry message = 1;       // Single message per response (streaming) | |
|     string error = 2;           // Error message if any | |
|     bool end_of_stream = 3;     // Indicates this is the final response | |
| } | |
|  | |
| ////////////////////////////////////////////////// | |
| // Partition range information messages | |
|  | |
| message GetPartitionRangeInfoRequest { | |
|     schema_pb.Topic topic = 1; | |
|     schema_pb.Partition partition = 2; | |
| } | |
|  | |
| message GetPartitionRangeInfoResponse { | |
|     // Offset range information | |
|     OffsetRangeInfo offset_range = 1; | |
|      | |
|     // Timestamp range information | |
|     TimestampRangeInfo timestamp_range = 2; | |
|      | |
|     // Future: ID range information (for ordered IDs, UUIDs, etc.) | |
|     // IdRangeInfo id_range = 3; | |
|      | |
|     // Partition metadata | |
|     int64 record_count = 10; | |
|     int64 active_subscriptions = 11; | |
|     string error = 12; | |
| } | |
|  | |
| message OffsetRangeInfo { | |
|     int64 earliest_offset = 1; | |
|     int64 latest_offset = 2; | |
|     int64 high_water_mark = 3; | |
| } | |
|  | |
| message TimestampRangeInfo { | |
|     int64 earliest_timestamp_ns = 1;  // Earliest message timestamp in nanoseconds | |
|     int64 latest_timestamp_ns = 2;    // Latest message timestamp in nanoseconds | |
| } | |
|  | |
| // Future extension for ID ranges | |
| // message IdRangeInfo { | |
| //     string earliest_id = 1; | |
| //     string latest_id = 2; | |
| //     string id_type = 3;  // "uuid", "sequential", "custom", etc. | |
| // } | |
|  | |
| // Removed Kafka Gateway Registration messages - no longer needed
 |