aboutsummaryrefslogtreecommitdiff
path: root/weed/mq/offset/subscriber.go
blob: d39932aae0e56eed1fc1b8120bdfc5dff72e7a4c (plain)
1
2
3
4
5
6
7
8
9
10
11
12
13
14
15
16
17
18
19
20
21
22
23
24
25
26
27
28
29
30
31
32
33
34
35
36
37
38
39
40
41
42
43
44
45
46
47
48
49
50
51
52
53
54
55
56
57
58
59
60
61
62
63
64
65
66
67
68
69
70
71
72
73
74
75
76
77
78
79
80
81
82
83
84
85
86
87
88
89
90
91
92
93
94
95
96
97
98
99
100
101
102
103
104
105
106
107
108
109
110
111
112
113
114
115
116
117
118
119
120
121
122
123
124
125
126
127
128
129
130
131
132
133
134
135
136
137
138
139
140
141
142
143
144
145
146
147
148
149
150
151
152
153
154
155
156
157
158
159
160
161
162
163
164
165
166
167
168
169
170
171
172
173
174
175
176
177
178
179
180
181
182
183
184
185
186
187
188
189
190
191
192
193
194
195
196
197
198
199
200
201
202
203
204
205
206
207
208
209
210
211
212
213
214
215
216
217
218
219
220
221
222
223
224
225
226
227
228
229
230
231
232
233
234
235
236
237
238
239
240
241
242
243
244
245
246
247
248
249
250
251
252
253
254
255
256
257
258
259
260
261
262
263
264
265
266
267
268
269
270
271
272
273
274
275
276
277
278
279
280
281
282
283
284
285
286
287
288
289
290
291
292
293
294
295
296
297
298
299
300
301
302
303
304
305
306
307
308
309
310
311
312
313
314
315
316
317
318
319
320
321
322
323
324
325
326
327
328
329
330
331
332
333
334
335
336
337
338
339
340
341
342
343
344
345
346
347
348
349
350
351
352
353
354
355
package offset

import (
	"fmt"
	"sync"

	"github.com/seaweedfs/seaweedfs/weed/pb/schema_pb"
)

// OffsetSubscriber handles offset-based subscription logic
type OffsetSubscriber struct {
	mu             sync.RWMutex
	offsetRegistry *PartitionOffsetRegistry
	subscriptions  map[string]*OffsetSubscription
}

// OffsetSubscription represents an active offset-based subscription
type OffsetSubscription struct {
	ID             string
	Namespace      string
	TopicName      string
	Partition      *schema_pb.Partition
	StartOffset    int64
	CurrentOffset  int64
	OffsetType     schema_pb.OffsetType
	IsActive       bool
	offsetRegistry *PartitionOffsetRegistry
}

// NewOffsetSubscriber creates a new offset-based subscriber
func NewOffsetSubscriber(offsetRegistry *PartitionOffsetRegistry) *OffsetSubscriber {
	return &OffsetSubscriber{
		offsetRegistry: offsetRegistry,
		subscriptions:  make(map[string]*OffsetSubscription),
	}
}

// CreateSubscription creates a new offset-based subscription
func (s *OffsetSubscriber) CreateSubscription(
	subscriptionID string,
	namespace, topicName string,
	partition *schema_pb.Partition,
	offsetType schema_pb.OffsetType,
	startOffset int64,
) (*OffsetSubscription, error) {

	s.mu.Lock()
	defer s.mu.Unlock()

	// Check if subscription already exists
	if _, exists := s.subscriptions[subscriptionID]; exists {
		return nil, fmt.Errorf("subscription %s already exists", subscriptionID)
	}

	// Resolve the actual start offset based on type
	actualStartOffset, err := s.resolveStartOffset(namespace, topicName, partition, offsetType, startOffset)
	if err != nil {
		return nil, fmt.Errorf("failed to resolve start offset: %w", err)
	}

	subscription := &OffsetSubscription{
		ID:             subscriptionID,
		Namespace:      namespace,
		TopicName:      topicName,
		Partition:      partition,
		StartOffset:    actualStartOffset,
		CurrentOffset:  actualStartOffset,
		OffsetType:     offsetType,
		IsActive:       true,
		offsetRegistry: s.offsetRegistry,
	}

	s.subscriptions[subscriptionID] = subscription
	return subscription, nil
}

// GetSubscription retrieves an existing subscription
func (s *OffsetSubscriber) GetSubscription(subscriptionID string) (*OffsetSubscription, error) {
	s.mu.RLock()
	defer s.mu.RUnlock()

	subscription, exists := s.subscriptions[subscriptionID]
	if !exists {
		return nil, fmt.Errorf("subscription %s not found", subscriptionID)
	}

	return subscription, nil
}

// CloseSubscription closes and removes a subscription
func (s *OffsetSubscriber) CloseSubscription(subscriptionID string) error {
	s.mu.Lock()
	defer s.mu.Unlock()

	subscription, exists := s.subscriptions[subscriptionID]
	if !exists {
		return fmt.Errorf("subscription %s not found", subscriptionID)
	}

	subscription.IsActive = false
	delete(s.subscriptions, subscriptionID)
	return nil
}

// resolveStartOffset resolves the actual start offset based on OffsetType
func (s *OffsetSubscriber) resolveStartOffset(
	namespace, topicName string,
	partition *schema_pb.Partition,
	offsetType schema_pb.OffsetType,
	requestedOffset int64,
) (int64, error) {

	switch offsetType {
	case schema_pb.OffsetType_EXACT_OFFSET:
		// Validate that the requested offset exists
		return s.validateAndGetOffset(namespace, topicName, partition, requestedOffset)

	case schema_pb.OffsetType_RESET_TO_OFFSET:
		// Use the requested offset, even if it doesn't exist yet
		return requestedOffset, nil

	case schema_pb.OffsetType_RESET_TO_EARLIEST:
		// Start from offset 0
		return 0, nil

	case schema_pb.OffsetType_RESET_TO_LATEST:
		// Start from the current high water mark
		hwm, err := s.offsetRegistry.GetHighWaterMark(namespace, topicName, partition)
		if err != nil {
			return 0, err
		}
		return hwm, nil

	case schema_pb.OffsetType_RESUME_OR_EARLIEST:
		// Try to resume from a saved position, fallback to earliest
		// For now, just use earliest (consumer group position tracking will be added later)
		return 0, nil

	case schema_pb.OffsetType_RESUME_OR_LATEST:
		// Try to resume from a saved position, fallback to latest
		// For now, just use latest
		hwm, err := s.offsetRegistry.GetHighWaterMark(namespace, topicName, partition)
		if err != nil {
			return 0, err
		}
		return hwm, nil

	default:
		return 0, fmt.Errorf("unsupported offset type: %v", offsetType)
	}
}

// validateAndGetOffset validates that an offset exists and returns it
func (s *OffsetSubscriber) validateAndGetOffset(namespace, topicName string, partition *schema_pb.Partition, offset int64) (int64, error) {
	if offset < 0 {
		return 0, fmt.Errorf("offset cannot be negative: %d", offset)
	}

	// Get the current high water mark
	hwm, err := s.offsetRegistry.GetHighWaterMark(namespace, topicName, partition)
	if err != nil {
		return 0, fmt.Errorf("failed to get high water mark: %w", err)
	}

	// Check if offset is within valid range
	if offset >= hwm {
		return 0, fmt.Errorf("offset %d is beyond high water mark %d", offset, hwm)
	}

	return offset, nil
}

// SeekToOffset seeks a subscription to a specific offset
func (sub *OffsetSubscription) SeekToOffset(offset int64) error {
	if !sub.IsActive {
		return fmt.Errorf("subscription is not active")
	}

	// Validate the offset
	if offset < 0 {
		return fmt.Errorf("offset cannot be negative: %d", offset)
	}

	hwm, err := sub.offsetRegistry.GetHighWaterMark(sub.Namespace, sub.TopicName, sub.Partition)
	if err != nil {
		return fmt.Errorf("failed to get high water mark: %w", err)
	}

	if offset > hwm {
		return fmt.Errorf("offset %d is beyond high water mark %d", offset, hwm)
	}

	sub.CurrentOffset = offset
	return nil
}

// GetNextOffset returns the next offset to read
func (sub *OffsetSubscription) GetNextOffset() int64 {
	return sub.CurrentOffset
}

// AdvanceOffset advances the subscription to the next offset
func (sub *OffsetSubscription) AdvanceOffset() {
	sub.CurrentOffset++
}

// GetLag returns the lag between current position and high water mark
func (sub *OffsetSubscription) GetLag() (int64, error) {
	if !sub.IsActive {
		return 0, fmt.Errorf("subscription is not active")
	}

	hwm, err := sub.offsetRegistry.GetHighWaterMark(sub.Namespace, sub.TopicName, sub.Partition)
	if err != nil {
		return 0, fmt.Errorf("failed to get high water mark: %w", err)
	}

	lag := hwm - sub.CurrentOffset
	if lag < 0 {
		lag = 0
	}

	return lag, nil
}

// IsAtEnd checks if the subscription has reached the end of available data
func (sub *OffsetSubscription) IsAtEnd() (bool, error) {
	if !sub.IsActive {
		return true, fmt.Errorf("subscription is not active")
	}

	hwm, err := sub.offsetRegistry.GetHighWaterMark(sub.Namespace, sub.TopicName, sub.Partition)
	if err != nil {
		return false, fmt.Errorf("failed to get high water mark: %w", err)
	}

	return sub.CurrentOffset >= hwm, nil
}

// OffsetRange represents a range of offsets
type OffsetRange struct {
	StartOffset int64
	EndOffset   int64
	Count       int64
}

// GetOffsetRange returns a range of offsets for batch reading
func (sub *OffsetSubscription) GetOffsetRange(maxCount int64) (*OffsetRange, error) {
	if !sub.IsActive {
		return nil, fmt.Errorf("subscription is not active")
	}

	hwm, err := sub.offsetRegistry.GetHighWaterMark(sub.Namespace, sub.TopicName, sub.Partition)
	if err != nil {
		return nil, fmt.Errorf("failed to get high water mark: %w", err)
	}

	startOffset := sub.CurrentOffset
	endOffset := startOffset + maxCount - 1

	// Don't go beyond high water mark
	if endOffset >= hwm {
		endOffset = hwm - 1
	}

	// If start is already at or beyond HWM, return empty range
	if startOffset >= hwm {
		return &OffsetRange{
			StartOffset: startOffset,
			EndOffset:   startOffset - 1, // Empty range
			Count:       0,
		}, nil
	}

	count := endOffset - startOffset + 1
	return &OffsetRange{
		StartOffset: startOffset,
		EndOffset:   endOffset,
		Count:       count,
	}, nil
}

// AdvanceOffsetBy advances the subscription by a specific number of offsets
func (sub *OffsetSubscription) AdvanceOffsetBy(count int64) {
	sub.CurrentOffset += count
}

// OffsetSeeker provides utilities for offset-based seeking
type OffsetSeeker struct {
	offsetRegistry *PartitionOffsetRegistry
}

// NewOffsetSeeker creates a new offset seeker
func NewOffsetSeeker(offsetRegistry *PartitionOffsetRegistry) *OffsetSeeker {
	return &OffsetSeeker{
		offsetRegistry: offsetRegistry,
	}
}

// SeekToTimestamp finds the offset closest to a given timestamp
// This bridges offset-based and timestamp-based seeking
func (seeker *OffsetSeeker) SeekToTimestamp(partition *schema_pb.Partition, timestamp int64) (int64, error) {
	// TODO: This requires integration with the storage layer to map timestamps to offsets
	// For now, return an error indicating this feature needs implementation
	return 0, fmt.Errorf("timestamp-to-offset mapping not implemented yet")
}

// ValidateOffsetRange validates that an offset range is valid
func (seeker *OffsetSeeker) ValidateOffsetRange(namespace, topicName string, partition *schema_pb.Partition, startOffset, endOffset int64) error {
	if startOffset < 0 {
		return fmt.Errorf("start offset cannot be negative: %d", startOffset)
	}

	if endOffset < startOffset {
		return fmt.Errorf("end offset %d cannot be less than start offset %d", endOffset, startOffset)
	}

	hwm, err := seeker.offsetRegistry.GetHighWaterMark(namespace, topicName, partition)
	if err != nil {
		return fmt.Errorf("failed to get high water mark: %w", err)
	}

	if startOffset >= hwm {
		return fmt.Errorf("start offset %d is beyond high water mark %d", startOffset, hwm)
	}

	if endOffset >= hwm {
		return fmt.Errorf("end offset %d is beyond high water mark %d", endOffset, hwm)
	}

	return nil
}

// GetAvailableOffsetRange returns the range of available offsets for a partition
func (seeker *OffsetSeeker) GetAvailableOffsetRange(namespace, topicName string, partition *schema_pb.Partition) (*OffsetRange, error) {
	hwm, err := seeker.offsetRegistry.GetHighWaterMark(namespace, topicName, partition)
	if err != nil {
		return nil, fmt.Errorf("failed to get high water mark: %w", err)
	}

	if hwm == 0 {
		// No data available
		return &OffsetRange{
			StartOffset: 0,
			EndOffset:   -1,
			Count:       0,
		}, nil
	}

	return &OffsetRange{
		StartOffset: 0,
		EndOffset:   hwm - 1,
		Count:       hwm,
	}, nil
}