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.
		
		
		
		
		
			
		
			
				
					
					
						
							509 lines
						
					
					
						
							16 KiB
						
					
					
				
			
		
		
		
			
			
			
		
		
	
	
							509 lines
						
					
					
						
							16 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) {
							 | 
						|
								    }
							 | 
						|
								    
							 | 
						|
								    // Stateless fetch API (Kafka-style) - request/response pattern
							 | 
						|
								    // This is the recommended API for Kafka gateway and other stateless clients
							 | 
						|
								    // No streaming, no session state - each request is completely independent
							 | 
						|
								    rpc FetchMessage (FetchMessageRequest) returns (FetchMessageResponse) {
							 | 
						|
								    }
							 | 
						|
								
							 | 
						|
								    // 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;
							 | 
						|
								    }
							 | 
						|
								    message SeekMessage {
							 | 
						|
								        int64 offset = 1;  // New offset to seek to
							 | 
						|
								        schema_pb.OffsetType offset_type = 2;  // EXACT_OFFSET, RESET_TO_LATEST, etc.
							 | 
						|
								    }
							 | 
						|
								    oneof message {
							 | 
						|
								        InitMessage init = 1;
							 | 
						|
								        AckMessage ack = 2;
							 | 
						|
								        SeekMessage seek = 3;
							 | 
						|
								    }
							 | 
						|
								}
							 | 
						|
								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;
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								//////////////////////////////////////////////////
							 | 
						|
								// Stateless Fetch API (Kafka-style)
							 | 
						|
								// Unlike SubscribeMessage which maintains long-lived Subscribe loops,
							 | 
						|
								// FetchMessage is completely stateless - each request is independent.
							 | 
						|
								// This eliminates concurrent access issues and stream corruption.
							 | 
						|
								//
							 | 
						|
								// Key differences from SubscribeMessage:
							 | 
						|
								// 1. Request/Response pattern (not streaming)
							 | 
						|
								// 2. No session state maintained
							 | 
						|
								// 3. Each fetch is independent
							 | 
						|
								// 4. Natural support for concurrent reads at different offsets
							 | 
						|
								// 5. Client manages offset tracking (like Kafka)
							 | 
						|
								//////////////////////////////////////////////////
							 | 
						|
								
							 | 
						|
								message FetchMessageRequest {
							 | 
						|
								    // Topic and partition to fetch from
							 | 
						|
								    schema_pb.Topic topic = 1;
							 | 
						|
								    schema_pb.Partition partition = 2;
							 | 
						|
								    
							 | 
						|
								    // Starting offset for this fetch
							 | 
						|
								    int64 start_offset = 3;
							 | 
						|
								    
							 | 
						|
								    // Maximum number of bytes to return (limit response size)
							 | 
						|
								    int32 max_bytes = 4;
							 | 
						|
								    
							 | 
						|
								    // Maximum number of messages to return
							 | 
						|
								    int32 max_messages = 5;
							 | 
						|
								    
							 | 
						|
								    // Maximum time to wait for data if partition is empty (milliseconds)
							 | 
						|
								    // 0 = return immediately, >0 = wait up to this long
							 | 
						|
								    int32 max_wait_ms = 6;
							 | 
						|
								    
							 | 
						|
								    // Minimum bytes before responding (0 = respond immediately)
							 | 
						|
								    // This allows batching for efficiency
							 | 
						|
								    int32 min_bytes = 7;
							 | 
						|
								    
							 | 
						|
								    // Consumer identity (for monitoring/debugging)
							 | 
						|
								    string consumer_group = 8;
							 | 
						|
								    string consumer_id = 9;
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								message FetchMessageResponse {
							 | 
						|
								    // Messages fetched (may be empty if no data available)
							 | 
						|
								    repeated DataMessage messages = 1;
							 | 
						|
								    
							 | 
						|
								    // Metadata about partition state
							 | 
						|
								    int64 high_water_mark = 2;  // Highest offset available
							 | 
						|
								    int64 log_start_offset = 3;  // Earliest offset available
							 | 
						|
								    bool end_of_partition = 4;   // True if no more data available
							 | 
						|
								    
							 | 
						|
								    // Error handling
							 | 
						|
								    string error = 5;
							 | 
						|
								    int32 error_code = 6;
							 | 
						|
								    
							 | 
						|
								    // Next offset to fetch (for client convenience)
							 | 
						|
								    // Client should fetch from this offset next
							 | 
						|
								    int64 next_offset = 7;
							 | 
						|
								}
							 | 
						|
								
							 | 
						|
								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
							 |