Add Unique Queue infrastructure and move TestPullRequests to this (#9856)
	
		
	
				
					
				
			* Upgrade levelqueue to version 0.2.0 This adds functionality for Unique Queues * Add UniqueQueue interface and functions to create them * Add UniqueQueue implementations * Move TestPullRequests over to use UniqueQueue * Reduce code duplication * Add bytefifos * Ensure invalid types are logged * Fix close race in PersistableChannelQueue Shutdowntokarchuk/v1.17
							parent
							
								
									b4914249ee
								
							
						
					
					
						commit
						2c903383b5
					
				@ -0,0 +1,61 @@ | 
				
			||||
// Copyright 2020 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
// ByteFIFO defines a FIFO that takes a byte array
 | 
				
			||||
type ByteFIFO interface { | 
				
			||||
	// Len returns the length of the fifo
 | 
				
			||||
	Len() int64 | 
				
			||||
	// PushFunc pushes data to the end of the fifo and calls the callback if it is added
 | 
				
			||||
	PushFunc(data []byte, fn func() error) error | 
				
			||||
	// Pop pops data from the start of the fifo
 | 
				
			||||
	Pop() ([]byte, error) | 
				
			||||
	// Close this fifo
 | 
				
			||||
	Close() error | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// UniqueByteFIFO defines a FIFO that Uniques its contents
 | 
				
			||||
type UniqueByteFIFO interface { | 
				
			||||
	ByteFIFO | 
				
			||||
	// Has returns whether the fifo contains this data
 | 
				
			||||
	Has(data []byte) (bool, error) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
var _ (ByteFIFO) = &DummyByteFIFO{} | 
				
			||||
 | 
				
			||||
// DummyByteFIFO represents a dummy fifo
 | 
				
			||||
type DummyByteFIFO struct{} | 
				
			||||
 | 
				
			||||
// PushFunc returns nil
 | 
				
			||||
func (*DummyByteFIFO) PushFunc(data []byte, fn func() error) error { | 
				
			||||
	return nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Pop returns nil
 | 
				
			||||
func (*DummyByteFIFO) Pop() ([]byte, error) { | 
				
			||||
	return []byte{}, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Close returns nil
 | 
				
			||||
func (*DummyByteFIFO) Close() error { | 
				
			||||
	return nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Len is always 0
 | 
				
			||||
func (*DummyByteFIFO) Len() int64 { | 
				
			||||
	return 0 | 
				
			||||
} | 
				
			||||
 | 
				
			||||
var _ (UniqueByteFIFO) = &DummyUniqueByteFIFO{} | 
				
			||||
 | 
				
			||||
// DummyUniqueByteFIFO represents a dummy unique fifo
 | 
				
			||||
type DummyUniqueByteFIFO struct { | 
				
			||||
	DummyByteFIFO | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has always returns false
 | 
				
			||||
func (*DummyUniqueByteFIFO) Has([]byte) (bool, error) { | 
				
			||||
	return false, nil | 
				
			||||
} | 
				
			||||
@ -0,0 +1,227 @@ | 
				
			||||
// Copyright 2020 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"context" | 
				
			||||
	"encoding/json" | 
				
			||||
	"fmt" | 
				
			||||
	"sync" | 
				
			||||
	"time" | 
				
			||||
 | 
				
			||||
	"code.gitea.io/gitea/modules/log" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// ByteFIFOQueueConfiguration is the configuration for a ByteFIFOQueue
 | 
				
			||||
type ByteFIFOQueueConfiguration struct { | 
				
			||||
	WorkerPoolConfiguration | 
				
			||||
	Workers int | 
				
			||||
	Name    string | 
				
			||||
} | 
				
			||||
 | 
				
			||||
var _ (Queue) = &ByteFIFOQueue{} | 
				
			||||
 | 
				
			||||
// ByteFIFOQueue is a Queue formed from a ByteFIFO and WorkerPool
 | 
				
			||||
type ByteFIFOQueue struct { | 
				
			||||
	*WorkerPool | 
				
			||||
	byteFIFO   ByteFIFO | 
				
			||||
	typ        Type | 
				
			||||
	closed     chan struct{} | 
				
			||||
	terminated chan struct{} | 
				
			||||
	exemplar   interface{} | 
				
			||||
	workers    int | 
				
			||||
	name       string | 
				
			||||
	lock       sync.Mutex | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewByteFIFOQueue creates a new ByteFIFOQueue
 | 
				
			||||
func NewByteFIFOQueue(typ Type, byteFIFO ByteFIFO, handle HandlerFunc, cfg, exemplar interface{}) (*ByteFIFOQueue, error) { | 
				
			||||
	configInterface, err := toConfig(ByteFIFOQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(ByteFIFOQueueConfiguration) | 
				
			||||
 | 
				
			||||
	return &ByteFIFOQueue{ | 
				
			||||
		WorkerPool: NewWorkerPool(handle, config.WorkerPoolConfiguration), | 
				
			||||
		byteFIFO:   byteFIFO, | 
				
			||||
		typ:        typ, | 
				
			||||
		closed:     make(chan struct{}), | 
				
			||||
		terminated: make(chan struct{}), | 
				
			||||
		exemplar:   exemplar, | 
				
			||||
		workers:    config.Workers, | 
				
			||||
		name:       config.Name, | 
				
			||||
	}, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Name returns the name of this queue
 | 
				
			||||
func (q *ByteFIFOQueue) Name() string { | 
				
			||||
	return q.name | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Push pushes data to the fifo
 | 
				
			||||
func (q *ByteFIFOQueue) Push(data Data) error { | 
				
			||||
	return q.PushFunc(data, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PushFunc pushes data to the fifo
 | 
				
			||||
func (q *ByteFIFOQueue) PushFunc(data Data, fn func() error) error { | 
				
			||||
	if !assignableTo(data, q.exemplar) { | 
				
			||||
		return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) | 
				
			||||
	} | 
				
			||||
	bs, err := json.Marshal(data) | 
				
			||||
	if err != nil { | 
				
			||||
		return err | 
				
			||||
	} | 
				
			||||
	return q.byteFIFO.PushFunc(bs, fn) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// IsEmpty checks if the queue is empty
 | 
				
			||||
func (q *ByteFIFOQueue) IsEmpty() bool { | 
				
			||||
	q.lock.Lock() | 
				
			||||
	defer q.lock.Unlock() | 
				
			||||
	if !q.WorkerPool.IsEmpty() { | 
				
			||||
		return false | 
				
			||||
	} | 
				
			||||
	return q.byteFIFO.Len() == 0 | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Run runs the bytefifo queue
 | 
				
			||||
func (q *ByteFIFOQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | 
				
			||||
	atShutdown(context.Background(), q.Shutdown) | 
				
			||||
	atTerminate(context.Background(), q.Terminate) | 
				
			||||
	log.Debug("%s: %s Starting", q.typ, q.name) | 
				
			||||
 | 
				
			||||
	go func() { | 
				
			||||
		_ = q.AddWorkers(q.workers, 0) | 
				
			||||
	}() | 
				
			||||
 | 
				
			||||
	go q.readToChan() | 
				
			||||
 | 
				
			||||
	log.Trace("%s: %s Waiting til closed", q.typ, q.name) | 
				
			||||
	<-q.closed | 
				
			||||
	log.Trace("%s: %s Waiting til done", q.typ, q.name) | 
				
			||||
	q.Wait() | 
				
			||||
 | 
				
			||||
	log.Trace("%s: %s Waiting til cleaned", q.typ, q.name) | 
				
			||||
	ctx, cancel := context.WithCancel(context.Background()) | 
				
			||||
	atTerminate(ctx, cancel) | 
				
			||||
	q.CleanUp(ctx) | 
				
			||||
	cancel() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
func (q *ByteFIFOQueue) readToChan() { | 
				
			||||
	for { | 
				
			||||
		select { | 
				
			||||
		case <-q.closed: | 
				
			||||
			// tell the pool to shutdown.
 | 
				
			||||
			q.cancel() | 
				
			||||
			return | 
				
			||||
		default: | 
				
			||||
			q.lock.Lock() | 
				
			||||
			bs, err := q.byteFIFO.Pop() | 
				
			||||
			if err != nil { | 
				
			||||
				q.lock.Unlock() | 
				
			||||
				log.Error("%s: %s Error on Pop: %v", q.typ, q.name, err) | 
				
			||||
				time.Sleep(time.Millisecond * 100) | 
				
			||||
				continue | 
				
			||||
			} | 
				
			||||
 | 
				
			||||
			if len(bs) == 0 { | 
				
			||||
				q.lock.Unlock() | 
				
			||||
				time.Sleep(time.Millisecond * 100) | 
				
			||||
				continue | 
				
			||||
			} | 
				
			||||
 | 
				
			||||
			data, err := unmarshalAs(bs, q.exemplar) | 
				
			||||
			if err != nil { | 
				
			||||
				log.Error("%s: %s Failed to unmarshal with error: %v", q.typ, q.name, err) | 
				
			||||
				q.lock.Unlock() | 
				
			||||
				time.Sleep(time.Millisecond * 100) | 
				
			||||
				continue | 
				
			||||
			} | 
				
			||||
 | 
				
			||||
			log.Trace("%s %s: Task found: %#v", q.typ, q.name, data) | 
				
			||||
			q.WorkerPool.Push(data) | 
				
			||||
			q.lock.Unlock() | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Shutdown processing from this queue
 | 
				
			||||
func (q *ByteFIFOQueue) Shutdown() { | 
				
			||||
	log.Trace("%s: %s Shutting down", q.typ, q.name) | 
				
			||||
	q.lock.Lock() | 
				
			||||
	select { | 
				
			||||
	case <-q.closed: | 
				
			||||
	default: | 
				
			||||
		close(q.closed) | 
				
			||||
	} | 
				
			||||
	q.lock.Unlock() | 
				
			||||
	log.Debug("%s: %s Shutdown", q.typ, q.name) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Terminate this queue and close the queue
 | 
				
			||||
func (q *ByteFIFOQueue) Terminate() { | 
				
			||||
	log.Trace("%s: %s Terminating", q.typ, q.name) | 
				
			||||
	q.Shutdown() | 
				
			||||
	q.lock.Lock() | 
				
			||||
	select { | 
				
			||||
	case <-q.terminated: | 
				
			||||
		q.lock.Unlock() | 
				
			||||
		return | 
				
			||||
	default: | 
				
			||||
	} | 
				
			||||
	close(q.terminated) | 
				
			||||
	q.lock.Unlock() | 
				
			||||
	if log.IsDebug() { | 
				
			||||
		log.Debug("%s: %s Closing with %d tasks left in queue", q.typ, q.name, q.byteFIFO.Len()) | 
				
			||||
	} | 
				
			||||
	if err := q.byteFIFO.Close(); err != nil { | 
				
			||||
		log.Error("Error whilst closing internal byte fifo in %s: %s: %v", q.typ, q.name, err) | 
				
			||||
	} | 
				
			||||
	log.Debug("%s: %s Terminated", q.typ, q.name) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
var _ (UniqueQueue) = &ByteFIFOUniqueQueue{} | 
				
			||||
 | 
				
			||||
// ByteFIFOUniqueQueue represents a UniqueQueue formed from a UniqueByteFifo
 | 
				
			||||
type ByteFIFOUniqueQueue struct { | 
				
			||||
	ByteFIFOQueue | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewByteFIFOUniqueQueue creates a new ByteFIFOUniqueQueue
 | 
				
			||||
func NewByteFIFOUniqueQueue(typ Type, byteFIFO UniqueByteFIFO, handle HandlerFunc, cfg, exemplar interface{}) (*ByteFIFOUniqueQueue, error) { | 
				
			||||
	configInterface, err := toConfig(ByteFIFOQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(ByteFIFOQueueConfiguration) | 
				
			||||
 | 
				
			||||
	return &ByteFIFOUniqueQueue{ | 
				
			||||
		ByteFIFOQueue: ByteFIFOQueue{ | 
				
			||||
			WorkerPool: NewWorkerPool(handle, config.WorkerPoolConfiguration), | 
				
			||||
			byteFIFO:   byteFIFO, | 
				
			||||
			typ:        typ, | 
				
			||||
			closed:     make(chan struct{}), | 
				
			||||
			terminated: make(chan struct{}), | 
				
			||||
			exemplar:   exemplar, | 
				
			||||
			workers:    config.Workers, | 
				
			||||
			name:       config.Name, | 
				
			||||
		}, | 
				
			||||
	}, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has checks if the provided data is in the queue
 | 
				
			||||
func (q *ByteFIFOUniqueQueue) Has(data Data) (bool, error) { | 
				
			||||
	if !assignableTo(data, q.exemplar) { | 
				
			||||
		return false, fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) | 
				
			||||
	} | 
				
			||||
	bs, err := json.Marshal(data) | 
				
			||||
	if err != nil { | 
				
			||||
		return false, err | 
				
			||||
	} | 
				
			||||
	return q.byteFIFO.(UniqueByteFIFO).Has(bs) | 
				
			||||
} | 
				
			||||
@ -0,0 +1,29 @@ | 
				
			||||
// Copyright 2020 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"fmt" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// UniqueQueue defines a queue which guarantees only one instance of same
 | 
				
			||||
// data is in the queue. Instances with same identity will be
 | 
				
			||||
// discarded if there is already one in the line.
 | 
				
			||||
//
 | 
				
			||||
// This queue is particularly useful for preventing duplicated task
 | 
				
			||||
// of same purpose - please note that this does not guarantee that a particular
 | 
				
			||||
// task cannot be processed twice or more at the same time. Uniqueness is
 | 
				
			||||
// only guaranteed whilst the task is waiting in the queue.
 | 
				
			||||
//
 | 
				
			||||
// Users of this queue should be careful to push only the identifier of the
 | 
				
			||||
// data
 | 
				
			||||
type UniqueQueue interface { | 
				
			||||
	Queue | 
				
			||||
	PushFunc(Data, func() error) error | 
				
			||||
	Has(Data) (bool, error) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// ErrAlreadyInQueue is returned when trying to push data to the queue that is already in the queue
 | 
				
			||||
var ErrAlreadyInQueue = fmt.Errorf("already in queue") | 
				
			||||
@ -0,0 +1,132 @@ | 
				
			||||
// Copyright 2020 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"context" | 
				
			||||
	"fmt" | 
				
			||||
	"sync" | 
				
			||||
 | 
				
			||||
	"code.gitea.io/gitea/modules/log" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// ChannelUniqueQueueType is the type for channel queue
 | 
				
			||||
const ChannelUniqueQueueType Type = "unique-channel" | 
				
			||||
 | 
				
			||||
// ChannelUniqueQueueConfiguration is the configuration for a ChannelUniqueQueue
 | 
				
			||||
type ChannelUniqueQueueConfiguration ChannelQueueConfiguration | 
				
			||||
 | 
				
			||||
// ChannelUniqueQueue implements UniqueQueue
 | 
				
			||||
//
 | 
				
			||||
// It is basically a thin wrapper around a WorkerPool but keeps a store of
 | 
				
			||||
// what has been pushed within a table.
 | 
				
			||||
//
 | 
				
			||||
// Please note that this Queue does not guarantee that a particular
 | 
				
			||||
// task cannot be processed twice or more at the same time. Uniqueness is
 | 
				
			||||
// only guaranteed whilst the task is waiting in the queue.
 | 
				
			||||
type ChannelUniqueQueue struct { | 
				
			||||
	*WorkerPool | 
				
			||||
	lock     sync.Mutex | 
				
			||||
	table    map[Data]bool | 
				
			||||
	exemplar interface{} | 
				
			||||
	workers  int | 
				
			||||
	name     string | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewChannelUniqueQueue create a memory channel queue
 | 
				
			||||
func NewChannelUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | 
				
			||||
	configInterface, err := toConfig(ChannelUniqueQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(ChannelUniqueQueueConfiguration) | 
				
			||||
	if config.BatchLength == 0 { | 
				
			||||
		config.BatchLength = 1 | 
				
			||||
	} | 
				
			||||
	queue := &ChannelUniqueQueue{ | 
				
			||||
		table:    map[Data]bool{}, | 
				
			||||
		exemplar: exemplar, | 
				
			||||
		workers:  config.Workers, | 
				
			||||
		name:     config.Name, | 
				
			||||
	} | 
				
			||||
	queue.WorkerPool = NewWorkerPool(func(data ...Data) { | 
				
			||||
		for _, datum := range data { | 
				
			||||
			queue.lock.Lock() | 
				
			||||
			delete(queue.table, datum) | 
				
			||||
			queue.lock.Unlock() | 
				
			||||
			handle(datum) | 
				
			||||
		} | 
				
			||||
	}, config.WorkerPoolConfiguration) | 
				
			||||
 | 
				
			||||
	queue.qid = GetManager().Add(queue, ChannelUniqueQueueType, config, exemplar) | 
				
			||||
	return queue, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Run starts to run the queue
 | 
				
			||||
func (q *ChannelUniqueQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | 
				
			||||
	atShutdown(context.Background(), func() { | 
				
			||||
		log.Warn("ChannelUniqueQueue: %s is not shutdownable!", q.name) | 
				
			||||
	}) | 
				
			||||
	atTerminate(context.Background(), func() { | 
				
			||||
		log.Warn("ChannelUniqueQueue: %s is not terminatable!", q.name) | 
				
			||||
	}) | 
				
			||||
	log.Debug("ChannelUniqueQueue: %s Starting", q.name) | 
				
			||||
	go func() { | 
				
			||||
		_ = q.AddWorkers(q.workers, 0) | 
				
			||||
	}() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Push will push data into the queue if the data is not already in the queue
 | 
				
			||||
func (q *ChannelUniqueQueue) Push(data Data) error { | 
				
			||||
	return q.PushFunc(data, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PushFunc will push data into the queue
 | 
				
			||||
func (q *ChannelUniqueQueue) PushFunc(data Data, fn func() error) error { | 
				
			||||
	if !assignableTo(data, q.exemplar) { | 
				
			||||
		return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in queue: %s", data, q.exemplar, q.name) | 
				
			||||
	} | 
				
			||||
	q.lock.Lock() | 
				
			||||
	locked := true | 
				
			||||
	defer func() { | 
				
			||||
		if locked { | 
				
			||||
			q.lock.Unlock() | 
				
			||||
		} | 
				
			||||
	}() | 
				
			||||
	if _, ok := q.table[data]; ok { | 
				
			||||
		return ErrAlreadyInQueue | 
				
			||||
	} | 
				
			||||
	// FIXME: We probably need to implement some sort of limit here
 | 
				
			||||
	// If the downstream queue blocks this table will grow without limit
 | 
				
			||||
	q.table[data] = true | 
				
			||||
	if fn != nil { | 
				
			||||
		err := fn() | 
				
			||||
		if err != nil { | 
				
			||||
			delete(q.table, data) | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
	locked = false | 
				
			||||
	q.lock.Unlock() | 
				
			||||
	q.WorkerPool.Push(data) | 
				
			||||
	return nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has checks if the data is in the queue
 | 
				
			||||
func (q *ChannelUniqueQueue) Has(data Data) (bool, error) { | 
				
			||||
	q.lock.Lock() | 
				
			||||
	defer q.lock.Unlock() | 
				
			||||
	_, has := q.table[data] | 
				
			||||
	return has, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Name returns the name of this queue
 | 
				
			||||
func (q *ChannelUniqueQueue) Name() string { | 
				
			||||
	return q.name | 
				
			||||
} | 
				
			||||
 | 
				
			||||
func init() { | 
				
			||||
	queuesMap[ChannelUniqueQueueType] = NewChannelUniqueQueue | 
				
			||||
} | 
				
			||||
@ -0,0 +1,104 @@ | 
				
			||||
// Copyright 2019 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"gitea.com/lunny/levelqueue" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// LevelUniqueQueueType is the type for level queue
 | 
				
			||||
const LevelUniqueQueueType Type = "unique-level" | 
				
			||||
 | 
				
			||||
// LevelUniqueQueueConfiguration is the configuration for a LevelUniqueQueue
 | 
				
			||||
type LevelUniqueQueueConfiguration struct { | 
				
			||||
	ByteFIFOQueueConfiguration | 
				
			||||
	DataDir string | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// LevelUniqueQueue implements a disk library queue
 | 
				
			||||
type LevelUniqueQueue struct { | 
				
			||||
	*ByteFIFOUniqueQueue | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewLevelUniqueQueue creates a ledis local queue
 | 
				
			||||
//
 | 
				
			||||
// Please note that this Queue does not guarantee that a particular
 | 
				
			||||
// task cannot be processed twice or more at the same time. Uniqueness is
 | 
				
			||||
// only guaranteed whilst the task is waiting in the queue.
 | 
				
			||||
func NewLevelUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | 
				
			||||
	configInterface, err := toConfig(LevelUniqueQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(LevelUniqueQueueConfiguration) | 
				
			||||
 | 
				
			||||
	byteFIFO, err := NewLevelUniqueQueueByteFIFO(config.DataDir) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	byteFIFOQueue, err := NewByteFIFOUniqueQueue(LevelUniqueQueueType, byteFIFO, handle, config.ByteFIFOQueueConfiguration, exemplar) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	queue := &LevelUniqueQueue{ | 
				
			||||
		ByteFIFOUniqueQueue: byteFIFOQueue, | 
				
			||||
	} | 
				
			||||
	queue.qid = GetManager().Add(queue, LevelUniqueQueueType, config, exemplar) | 
				
			||||
	return queue, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
var _ (UniqueByteFIFO) = &LevelUniqueQueueByteFIFO{} | 
				
			||||
 | 
				
			||||
// LevelUniqueQueueByteFIFO represents a ByteFIFO formed from a LevelUniqueQueue
 | 
				
			||||
type LevelUniqueQueueByteFIFO struct { | 
				
			||||
	internal *levelqueue.UniqueQueue | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewLevelUniqueQueueByteFIFO creates a new ByteFIFO formed from a LevelUniqueQueue
 | 
				
			||||
func NewLevelUniqueQueueByteFIFO(dataDir string) (*LevelUniqueQueueByteFIFO, error) { | 
				
			||||
	internal, err := levelqueue.OpenUnique(dataDir) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	return &LevelUniqueQueueByteFIFO{ | 
				
			||||
		internal: internal, | 
				
			||||
	}, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PushFunc pushes data to the end of the fifo and calls the callback if it is added
 | 
				
			||||
func (fifo *LevelUniqueQueueByteFIFO) PushFunc(data []byte, fn func() error) error { | 
				
			||||
	return fifo.internal.LPushFunc(data, fn) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Pop pops data from the start of the fifo
 | 
				
			||||
func (fifo *LevelUniqueQueueByteFIFO) Pop() ([]byte, error) { | 
				
			||||
	data, err := fifo.internal.RPop() | 
				
			||||
	if err != nil && err != levelqueue.ErrNotFound { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	return data, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Len returns the length of the fifo
 | 
				
			||||
func (fifo *LevelUniqueQueueByteFIFO) Len() int64 { | 
				
			||||
	return fifo.internal.Len() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has returns whether the fifo contains this data
 | 
				
			||||
func (fifo *LevelUniqueQueueByteFIFO) Has(data []byte) (bool, error) { | 
				
			||||
	return fifo.internal.Has(data) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Close this fifo
 | 
				
			||||
func (fifo *LevelUniqueQueueByteFIFO) Close() error { | 
				
			||||
	return fifo.internal.Close() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
func init() { | 
				
			||||
	queuesMap[LevelUniqueQueueType] = NewLevelUniqueQueue | 
				
			||||
} | 
				
			||||
@ -0,0 +1,241 @@ | 
				
			||||
// Copyright 2020 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"context" | 
				
			||||
	"sync" | 
				
			||||
	"time" | 
				
			||||
 | 
				
			||||
	"code.gitea.io/gitea/modules/log" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// PersistableChannelUniqueQueueType is the type for persistable queue
 | 
				
			||||
const PersistableChannelUniqueQueueType Type = "unique-persistable-channel" | 
				
			||||
 | 
				
			||||
// PersistableChannelUniqueQueueConfiguration is the configuration for a PersistableChannelUniqueQueue
 | 
				
			||||
type PersistableChannelUniqueQueueConfiguration struct { | 
				
			||||
	Name         string | 
				
			||||
	DataDir      string | 
				
			||||
	BatchLength  int | 
				
			||||
	QueueLength  int | 
				
			||||
	Timeout      time.Duration | 
				
			||||
	MaxAttempts  int | 
				
			||||
	Workers      int | 
				
			||||
	MaxWorkers   int | 
				
			||||
	BlockTimeout time.Duration | 
				
			||||
	BoostTimeout time.Duration | 
				
			||||
	BoostWorkers int | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PersistableChannelUniqueQueue wraps a channel queue and level queue together
 | 
				
			||||
//
 | 
				
			||||
// Please note that this Queue does not guarantee that a particular
 | 
				
			||||
// task cannot be processed twice or more at the same time. Uniqueness is
 | 
				
			||||
// only guaranteed whilst the task is waiting in the queue.
 | 
				
			||||
type PersistableChannelUniqueQueue struct { | 
				
			||||
	*ChannelUniqueQueue | 
				
			||||
	delayedStarter | 
				
			||||
	lock   sync.Mutex | 
				
			||||
	closed chan struct{} | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewPersistableChannelUniqueQueue creates a wrapped batched channel queue with persistable level queue backend when shutting down
 | 
				
			||||
// This differs from a wrapped queue in that the persistent queue is only used to persist at shutdown/terminate
 | 
				
			||||
func NewPersistableChannelUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | 
				
			||||
	configInterface, err := toConfig(PersistableChannelUniqueQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(PersistableChannelUniqueQueueConfiguration) | 
				
			||||
 | 
				
			||||
	channelUniqueQueue, err := NewChannelUniqueQueue(handle, ChannelUniqueQueueConfiguration{ | 
				
			||||
		WorkerPoolConfiguration: WorkerPoolConfiguration{ | 
				
			||||
			QueueLength:  config.QueueLength, | 
				
			||||
			BatchLength:  config.BatchLength, | 
				
			||||
			BlockTimeout: config.BlockTimeout, | 
				
			||||
			BoostTimeout: config.BoostTimeout, | 
				
			||||
			BoostWorkers: config.BoostWorkers, | 
				
			||||
			MaxWorkers:   config.MaxWorkers, | 
				
			||||
		}, | 
				
			||||
		Workers: config.Workers, | 
				
			||||
		Name:    config.Name + "-channel", | 
				
			||||
	}, exemplar) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	// the level backend only needs temporary workers to catch up with the previously dropped work
 | 
				
			||||
	levelCfg := LevelUniqueQueueConfiguration{ | 
				
			||||
		ByteFIFOQueueConfiguration: ByteFIFOQueueConfiguration{ | 
				
			||||
			WorkerPoolConfiguration: WorkerPoolConfiguration{ | 
				
			||||
				QueueLength:  config.QueueLength, | 
				
			||||
				BatchLength:  config.BatchLength, | 
				
			||||
				BlockTimeout: 0, | 
				
			||||
				BoostTimeout: 0, | 
				
			||||
				BoostWorkers: 0, | 
				
			||||
				MaxWorkers:   1, | 
				
			||||
			}, | 
				
			||||
			Workers: 1, | 
				
			||||
			Name:    config.Name + "-level", | 
				
			||||
		}, | 
				
			||||
		DataDir: config.DataDir, | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	queue := &PersistableChannelUniqueQueue{ | 
				
			||||
		ChannelUniqueQueue: channelUniqueQueue.(*ChannelUniqueQueue), | 
				
			||||
		closed:             make(chan struct{}), | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	levelQueue, err := NewLevelUniqueQueue(func(data ...Data) { | 
				
			||||
		for _, datum := range data { | 
				
			||||
			err := queue.Push(datum) | 
				
			||||
			if err != nil && err != ErrAlreadyInQueue { | 
				
			||||
				log.Error("Unable push to channelled queue: %v", err) | 
				
			||||
			} | 
				
			||||
		} | 
				
			||||
	}, levelCfg, exemplar) | 
				
			||||
	if err == nil { | 
				
			||||
		queue.delayedStarter = delayedStarter{ | 
				
			||||
			internal: levelQueue.(*LevelUniqueQueue), | 
				
			||||
			name:     config.Name, | 
				
			||||
		} | 
				
			||||
 | 
				
			||||
		_ = GetManager().Add(queue, PersistableChannelUniqueQueueType, config, exemplar) | 
				
			||||
		return queue, nil | 
				
			||||
	} | 
				
			||||
	if IsErrInvalidConfiguration(err) { | 
				
			||||
		// Retrying ain't gonna make this any better...
 | 
				
			||||
		return nil, ErrInvalidConfiguration{cfg: cfg} | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	queue.delayedStarter = delayedStarter{ | 
				
			||||
		cfg:         levelCfg, | 
				
			||||
		underlying:  LevelUniqueQueueType, | 
				
			||||
		timeout:     config.Timeout, | 
				
			||||
		maxAttempts: config.MaxAttempts, | 
				
			||||
		name:        config.Name, | 
				
			||||
	} | 
				
			||||
	_ = GetManager().Add(queue, PersistableChannelUniqueQueueType, config, exemplar) | 
				
			||||
	return queue, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Name returns the name of this queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Name() string { | 
				
			||||
	return q.delayedStarter.name | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Push will push the indexer data to queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Push(data Data) error { | 
				
			||||
	return q.PushFunc(data, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PushFunc will push the indexer data to queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) PushFunc(data Data, fn func() error) error { | 
				
			||||
	select { | 
				
			||||
	case <-q.closed: | 
				
			||||
		return q.internal.(UniqueQueue).PushFunc(data, fn) | 
				
			||||
	default: | 
				
			||||
		return q.ChannelUniqueQueue.PushFunc(data, fn) | 
				
			||||
	} | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has will test if the queue has the data
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Has(data Data) (bool, error) { | 
				
			||||
	// This is more difficult...
 | 
				
			||||
	has, err := q.ChannelUniqueQueue.Has(data) | 
				
			||||
	if err != nil || has { | 
				
			||||
		return has, err | 
				
			||||
	} | 
				
			||||
	return q.internal.(UniqueQueue).Has(data) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Run starts to run the queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | 
				
			||||
	log.Debug("PersistableChannelUniqueQueue: %s Starting", q.delayedStarter.name) | 
				
			||||
 | 
				
			||||
	q.lock.Lock() | 
				
			||||
	if q.internal == nil { | 
				
			||||
		err := q.setInternal(atShutdown, func(data ...Data) { | 
				
			||||
			for _, datum := range data { | 
				
			||||
				err := q.Push(datum) | 
				
			||||
				if err != nil && err != ErrAlreadyInQueue { | 
				
			||||
					log.Error("Unable push to channelled queue: %v", err) | 
				
			||||
				} | 
				
			||||
			} | 
				
			||||
		}, q.exemplar) | 
				
			||||
		q.lock.Unlock() | 
				
			||||
		if err != nil { | 
				
			||||
			log.Fatal("Unable to create internal queue for %s Error: %v", q.Name(), err) | 
				
			||||
			return | 
				
			||||
		} | 
				
			||||
	} else { | 
				
			||||
		q.lock.Unlock() | 
				
			||||
	} | 
				
			||||
	atShutdown(context.Background(), q.Shutdown) | 
				
			||||
	atTerminate(context.Background(), q.Terminate) | 
				
			||||
 | 
				
			||||
	// Just run the level queue - we shut it down later
 | 
				
			||||
	go q.internal.Run(func(_ context.Context, _ func()) {}, func(_ context.Context, _ func()) {}) | 
				
			||||
 | 
				
			||||
	go func() { | 
				
			||||
		_ = q.ChannelUniqueQueue.AddWorkers(q.workers, 0) | 
				
			||||
	}() | 
				
			||||
 | 
				
			||||
	log.Trace("PersistableChannelUniqueQueue: %s Waiting til closed", q.delayedStarter.name) | 
				
			||||
	<-q.closed | 
				
			||||
	log.Trace("PersistableChannelUniqueQueue: %s Cancelling pools", q.delayedStarter.name) | 
				
			||||
	q.internal.(*LevelUniqueQueue).cancel() | 
				
			||||
	q.ChannelUniqueQueue.cancel() | 
				
			||||
	log.Trace("PersistableChannelUniqueQueue: %s Waiting til done", q.delayedStarter.name) | 
				
			||||
	q.ChannelUniqueQueue.Wait() | 
				
			||||
	q.internal.(*LevelUniqueQueue).Wait() | 
				
			||||
	// Redirect all remaining data in the chan to the internal channel
 | 
				
			||||
	go func() { | 
				
			||||
		log.Trace("PersistableChannelUniqueQueue: %s Redirecting remaining data", q.delayedStarter.name) | 
				
			||||
		for data := range q.ChannelUniqueQueue.dataChan { | 
				
			||||
			_ = q.internal.Push(data) | 
				
			||||
		} | 
				
			||||
		log.Trace("PersistableChannelUniqueQueue: %s Done Redirecting remaining data", q.delayedStarter.name) | 
				
			||||
	}() | 
				
			||||
	log.Trace("PersistableChannelUniqueQueue: %s Done main loop", q.delayedStarter.name) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Flush flushes the queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Flush(timeout time.Duration) error { | 
				
			||||
	return q.ChannelUniqueQueue.Flush(timeout) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Shutdown processing this queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Shutdown() { | 
				
			||||
	log.Trace("PersistableChannelUniqueQueue: %s Shutting down", q.delayedStarter.name) | 
				
			||||
	q.lock.Lock() | 
				
			||||
	defer q.lock.Unlock() | 
				
			||||
	select { | 
				
			||||
	case <-q.closed: | 
				
			||||
	default: | 
				
			||||
		if q.internal != nil { | 
				
			||||
			q.internal.(*LevelUniqueQueue).Shutdown() | 
				
			||||
		} | 
				
			||||
		close(q.closed) | 
				
			||||
	} | 
				
			||||
	log.Debug("PersistableChannelUniqueQueue: %s Shutdown", q.delayedStarter.name) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Terminate this queue and close the queue
 | 
				
			||||
func (q *PersistableChannelUniqueQueue) Terminate() { | 
				
			||||
	log.Trace("PersistableChannelUniqueQueue: %s Terminating", q.delayedStarter.name) | 
				
			||||
	q.Shutdown() | 
				
			||||
	q.lock.Lock() | 
				
			||||
	defer q.lock.Unlock() | 
				
			||||
	if q.internal != nil { | 
				
			||||
		q.internal.(*LevelUniqueQueue).Terminate() | 
				
			||||
	} | 
				
			||||
	log.Debug("PersistableChannelUniqueQueue: %s Terminated", q.delayedStarter.name) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
func init() { | 
				
			||||
	queuesMap[PersistableChannelUniqueQueueType] = NewPersistableChannelUniqueQueue | 
				
			||||
} | 
				
			||||
@ -0,0 +1,124 @@ | 
				
			||||
// Copyright 2019 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
// RedisUniqueQueueType is the type for redis queue
 | 
				
			||||
const RedisUniqueQueueType Type = "unique-redis" | 
				
			||||
 | 
				
			||||
// RedisUniqueQueue redis queue
 | 
				
			||||
type RedisUniqueQueue struct { | 
				
			||||
	*ByteFIFOUniqueQueue | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// RedisUniqueQueueConfiguration is the configuration for the redis queue
 | 
				
			||||
type RedisUniqueQueueConfiguration struct { | 
				
			||||
	ByteFIFOQueueConfiguration | 
				
			||||
	RedisUniqueByteFIFOConfiguration | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewRedisUniqueQueue creates single redis or cluster redis queue.
 | 
				
			||||
//
 | 
				
			||||
// Please note that this Queue does not guarantee that a particular
 | 
				
			||||
// task cannot be processed twice or more at the same time. Uniqueness is
 | 
				
			||||
// only guaranteed whilst the task is waiting in the queue.
 | 
				
			||||
func NewRedisUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | 
				
			||||
	configInterface, err := toConfig(RedisUniqueQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(RedisUniqueQueueConfiguration) | 
				
			||||
 | 
				
			||||
	byteFIFO, err := NewRedisUniqueByteFIFO(config.RedisUniqueByteFIFOConfiguration) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	if len(byteFIFO.setName) == 0 { | 
				
			||||
		byteFIFO.setName = byteFIFO.queueName + "_unique" | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	byteFIFOQueue, err := NewByteFIFOUniqueQueue(RedisUniqueQueueType, byteFIFO, handle, config.ByteFIFOQueueConfiguration, exemplar) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	queue := &RedisUniqueQueue{ | 
				
			||||
		ByteFIFOUniqueQueue: byteFIFOQueue, | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	queue.qid = GetManager().Add(queue, RedisUniqueQueueType, config, exemplar) | 
				
			||||
 | 
				
			||||
	return queue, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
var _ (UniqueByteFIFO) = &RedisUniqueByteFIFO{} | 
				
			||||
 | 
				
			||||
// RedisUniqueByteFIFO represents a UniqueByteFIFO formed from a redisClient
 | 
				
			||||
type RedisUniqueByteFIFO struct { | 
				
			||||
	RedisByteFIFO | 
				
			||||
	setName string | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// RedisUniqueByteFIFOConfiguration is the configuration for the RedisUniqueByteFIFO
 | 
				
			||||
type RedisUniqueByteFIFOConfiguration struct { | 
				
			||||
	RedisByteFIFOConfiguration | 
				
			||||
	SetName string | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewRedisUniqueByteFIFO creates a UniqueByteFIFO formed from a redisClient
 | 
				
			||||
func NewRedisUniqueByteFIFO(config RedisUniqueByteFIFOConfiguration) (*RedisUniqueByteFIFO, error) { | 
				
			||||
	internal, err := NewRedisByteFIFO(config.RedisByteFIFOConfiguration) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	fifo := &RedisUniqueByteFIFO{ | 
				
			||||
		RedisByteFIFO: *internal, | 
				
			||||
		setName:       config.SetName, | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	return fifo, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PushFunc pushes data to the end of the fifo and calls the callback if it is added
 | 
				
			||||
func (fifo *RedisUniqueByteFIFO) PushFunc(data []byte, fn func() error) error { | 
				
			||||
	added, err := fifo.client.SAdd(fifo.setName, data).Result() | 
				
			||||
	if err != nil { | 
				
			||||
		return err | 
				
			||||
	} | 
				
			||||
	if added == 0 { | 
				
			||||
		return ErrAlreadyInQueue | 
				
			||||
	} | 
				
			||||
	if fn != nil { | 
				
			||||
		if err := fn(); err != nil { | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
	return fifo.client.RPush(fifo.queueName, data).Err() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Pop pops data from the start of the fifo
 | 
				
			||||
func (fifo *RedisUniqueByteFIFO) Pop() ([]byte, error) { | 
				
			||||
	data, err := fifo.client.LPop(fifo.queueName).Bytes() | 
				
			||||
	if err != nil { | 
				
			||||
		return data, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	if len(data) == 0 { | 
				
			||||
		return data, nil | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	err = fifo.client.SRem(fifo.setName, data).Err() | 
				
			||||
	return data, err | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has returns whether the fifo contains this data
 | 
				
			||||
func (fifo *RedisUniqueByteFIFO) Has(data []byte) (bool, error) { | 
				
			||||
	return fifo.client.SIsMember(fifo.setName, data).Result() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
func init() { | 
				
			||||
	queuesMap[RedisUniqueQueueType] = NewRedisUniqueQueue | 
				
			||||
} | 
				
			||||
@ -0,0 +1,172 @@ | 
				
			||||
// Copyright 2020 The Gitea Authors. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package queue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"fmt" | 
				
			||||
	"sync" | 
				
			||||
	"time" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// WrappedUniqueQueueType is the type for a wrapped delayed starting queue
 | 
				
			||||
const WrappedUniqueQueueType Type = "unique-wrapped" | 
				
			||||
 | 
				
			||||
// WrappedUniqueQueueConfiguration is the configuration for a WrappedUniqueQueue
 | 
				
			||||
type WrappedUniqueQueueConfiguration struct { | 
				
			||||
	Underlying  Type | 
				
			||||
	Timeout     time.Duration | 
				
			||||
	MaxAttempts int | 
				
			||||
	Config      interface{} | 
				
			||||
	QueueLength int | 
				
			||||
	Name        string | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// WrappedUniqueQueue wraps a delayed starting unique queue
 | 
				
			||||
type WrappedUniqueQueue struct { | 
				
			||||
	*WrappedQueue | 
				
			||||
	table map[Data]bool | 
				
			||||
	tlock sync.Mutex | 
				
			||||
	ready bool | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewWrappedUniqueQueue will attempt to create a unique queue of the provided type,
 | 
				
			||||
// but if there is a problem creating this queue it will instead create
 | 
				
			||||
// a WrappedUniqueQueue with delayed startup of the queue instead and a
 | 
				
			||||
// channel which will be redirected to the queue
 | 
				
			||||
//
 | 
				
			||||
// Please note that this Queue does not guarantee that a particular
 | 
				
			||||
// task cannot be processed twice or more at the same time. Uniqueness is
 | 
				
			||||
// only guaranteed whilst the task is waiting in the queue.
 | 
				
			||||
func NewWrappedUniqueQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | 
				
			||||
	configInterface, err := toConfig(WrappedUniqueQueueConfiguration{}, cfg) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	config := configInterface.(WrappedUniqueQueueConfiguration) | 
				
			||||
 | 
				
			||||
	queue, err := NewQueue(config.Underlying, handle, config.Config, exemplar) | 
				
			||||
	if err == nil { | 
				
			||||
		// Just return the queue there is no need to wrap
 | 
				
			||||
		return queue, nil | 
				
			||||
	} | 
				
			||||
	if IsErrInvalidConfiguration(err) { | 
				
			||||
		// Retrying ain't gonna make this any better...
 | 
				
			||||
		return nil, ErrInvalidConfiguration{cfg: cfg} | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	wrapped := &WrappedUniqueQueue{ | 
				
			||||
		WrappedQueue: &WrappedQueue{ | 
				
			||||
			channel:  make(chan Data, config.QueueLength), | 
				
			||||
			exemplar: exemplar, | 
				
			||||
			delayedStarter: delayedStarter{ | 
				
			||||
				cfg:         config.Config, | 
				
			||||
				underlying:  config.Underlying, | 
				
			||||
				timeout:     config.Timeout, | 
				
			||||
				maxAttempts: config.MaxAttempts, | 
				
			||||
				name:        config.Name, | 
				
			||||
			}, | 
				
			||||
		}, | 
				
			||||
		table: map[Data]bool{}, | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	// wrapped.handle is passed to the delayedStarting internal queue and is run to handle
 | 
				
			||||
	// data passed to
 | 
				
			||||
	wrapped.handle = func(data ...Data) { | 
				
			||||
		for _, datum := range data { | 
				
			||||
			wrapped.tlock.Lock() | 
				
			||||
			if !wrapped.ready { | 
				
			||||
				delete(wrapped.table, data) | 
				
			||||
				// If our table is empty all of the requests we have buffered between the
 | 
				
			||||
				// wrapper queue starting and the internal queue starting have been handled.
 | 
				
			||||
				// We can stop buffering requests in our local table and just pass Push
 | 
				
			||||
				// direct to the internal queue
 | 
				
			||||
				if len(wrapped.table) == 0 { | 
				
			||||
					wrapped.ready = true | 
				
			||||
				} | 
				
			||||
			} | 
				
			||||
			wrapped.tlock.Unlock() | 
				
			||||
			handle(datum) | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
	_ = GetManager().Add(queue, WrappedUniqueQueueType, config, exemplar) | 
				
			||||
	return wrapped, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Push will push the data to the internal channel checking it against the exemplar
 | 
				
			||||
func (q *WrappedUniqueQueue) Push(data Data) error { | 
				
			||||
	return q.PushFunc(data, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// PushFunc will push the data to the internal channel checking it against the exemplar
 | 
				
			||||
func (q *WrappedUniqueQueue) PushFunc(data Data, fn func() error) error { | 
				
			||||
	if !assignableTo(data, q.exemplar) { | 
				
			||||
		return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	q.tlock.Lock() | 
				
			||||
	if q.ready { | 
				
			||||
		// ready means our table is empty and all of the requests we have buffered between the
 | 
				
			||||
		// wrapper queue starting and the internal queue starting have been handled.
 | 
				
			||||
		// We can stop buffering requests in our local table and just pass Push
 | 
				
			||||
		// direct to the internal queue
 | 
				
			||||
		q.tlock.Unlock() | 
				
			||||
		return q.internal.(UniqueQueue).PushFunc(data, fn) | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	locked := true | 
				
			||||
	defer func() { | 
				
			||||
		if locked { | 
				
			||||
			q.tlock.Unlock() | 
				
			||||
		} | 
				
			||||
	}() | 
				
			||||
	if _, ok := q.table[data]; ok { | 
				
			||||
		return ErrAlreadyInQueue | 
				
			||||
	} | 
				
			||||
	// FIXME: We probably need to implement some sort of limit here
 | 
				
			||||
	// If the downstream queue blocks this table will grow without limit
 | 
				
			||||
	q.table[data] = true | 
				
			||||
	if fn != nil { | 
				
			||||
		err := fn() | 
				
			||||
		if err != nil { | 
				
			||||
			delete(q.table, data) | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
	locked = false | 
				
			||||
	q.tlock.Unlock() | 
				
			||||
 | 
				
			||||
	q.channel <- data | 
				
			||||
	return nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has checks if the data is in the queue
 | 
				
			||||
func (q *WrappedUniqueQueue) Has(data Data) (bool, error) { | 
				
			||||
	q.tlock.Lock() | 
				
			||||
	defer q.tlock.Unlock() | 
				
			||||
	if q.ready { | 
				
			||||
		return q.internal.(UniqueQueue).Has(data) | 
				
			||||
	} | 
				
			||||
	_, has := q.table[data] | 
				
			||||
	return has, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// IsEmpty checks whether the queue is empty
 | 
				
			||||
func (q *WrappedUniqueQueue) IsEmpty() bool { | 
				
			||||
	q.tlock.Lock() | 
				
			||||
	if len(q.table) > 0 { | 
				
			||||
		q.tlock.Unlock() | 
				
			||||
		return false | 
				
			||||
	} | 
				
			||||
	if q.ready { | 
				
			||||
		q.tlock.Unlock() | 
				
			||||
		return q.internal.IsEmpty() | 
				
			||||
	} | 
				
			||||
	q.tlock.Unlock() | 
				
			||||
	return false | 
				
			||||
} | 
				
			||||
 | 
				
			||||
func init() { | 
				
			||||
	queuesMap[WrappedUniqueQueueType] = NewWrappedUniqueQueue | 
				
			||||
} | 
				
			||||
@ -1,3 +1,7 @@ | 
				
			||||
queue/ | 
				
			||||
queue_pop/ | 
				
			||||
queue_push/ | 
				
			||||
uniquequeue/ | 
				
			||||
uniquequeue_pop/ | 
				
			||||
uniquequeue_push/ | 
				
			||||
set/ | 
				
			||||
 | 
				
			||||
@ -0,0 +1,110 @@ | 
				
			||||
// Copyright 2020 Andrew Thornton. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package levelqueue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"sync" | 
				
			||||
 | 
				
			||||
	"github.com/syndtr/goleveldb/leveldb" | 
				
			||||
	"github.com/syndtr/goleveldb/leveldb/util" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
const ( | 
				
			||||
	setPrefixStr = "set" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// Set defines a set struct
 | 
				
			||||
type Set struct { | 
				
			||||
	db                *leveldb.DB | 
				
			||||
	closeUnderlyingDB bool | 
				
			||||
	lock              sync.Mutex | 
				
			||||
	prefix            []byte | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// OpenSet opens a set from the db path or creates a set if it doesn't exist.
 | 
				
			||||
// The keys will be prefixed with "set-" by default
 | 
				
			||||
func OpenSet(dataDir string) (*Set, error) { | 
				
			||||
	db, err := leveldb.OpenFile(dataDir, nil) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	return NewSet(db, []byte(setPrefixStr), true) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewSet creates a set from a db. The keys will be prefixed with prefix
 | 
				
			||||
// and at close the db will be closed as per closeUnderlyingDB
 | 
				
			||||
func NewSet(db *leveldb.DB, prefix []byte, closeUnderlyingDB bool) (*Set, error) { | 
				
			||||
	set := &Set{ | 
				
			||||
		db:                db, | 
				
			||||
		closeUnderlyingDB: closeUnderlyingDB, | 
				
			||||
	} | 
				
			||||
	set.prefix = make([]byte, len(prefix)) | 
				
			||||
	copy(set.prefix, prefix) | 
				
			||||
 | 
				
			||||
	return set, nil | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Add adds a member string to a key set, returns true if the member was not already present
 | 
				
			||||
func (set *Set) Add(value []byte) (bool, error) { | 
				
			||||
	set.lock.Lock() | 
				
			||||
	defer set.lock.Unlock() | 
				
			||||
	setKey := withPrefix(set.prefix, value) | 
				
			||||
	has, err := set.db.Has(setKey, nil) | 
				
			||||
	if err != nil || has { | 
				
			||||
		return !has, err | 
				
			||||
	} | 
				
			||||
	return !has, set.db.Put(setKey, []byte(""), nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Members returns the current members of the set
 | 
				
			||||
func (set *Set) Members() ([][]byte, error) { | 
				
			||||
	set.lock.Lock() | 
				
			||||
	defer set.lock.Unlock() | 
				
			||||
	var members [][]byte | 
				
			||||
	prefix := withPrefix(set.prefix, []byte{}) | 
				
			||||
	iter := set.db.NewIterator(util.BytesPrefix(prefix), nil) | 
				
			||||
	for iter.Next() { | 
				
			||||
		slice := iter.Key()[len(prefix):] | 
				
			||||
		value := make([]byte, len(slice)) | 
				
			||||
		copy(value, slice) | 
				
			||||
		members = append(members, value) | 
				
			||||
	} | 
				
			||||
	iter.Release() | 
				
			||||
	return members, iter.Error() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has returns if the member is in the set
 | 
				
			||||
func (set *Set) Has(value []byte) (bool, error) { | 
				
			||||
	set.lock.Lock() | 
				
			||||
	defer set.lock.Unlock() | 
				
			||||
	setKey := withPrefix(set.prefix, value) | 
				
			||||
 | 
				
			||||
	return set.db.Has(setKey, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Remove removes a member from the set, returns true if the member was present
 | 
				
			||||
func (set *Set) Remove(value []byte) (bool, error) { | 
				
			||||
	set.lock.Lock() | 
				
			||||
	defer set.lock.Unlock() | 
				
			||||
	setKey := withPrefix(set.prefix, value) | 
				
			||||
 | 
				
			||||
	has, err := set.db.Has(setKey, nil) | 
				
			||||
	if err != nil || !has { | 
				
			||||
		return has, err | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	return has, set.db.Delete(setKey, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Close closes the set (and the underlying db if set to closeUnderlyingDB)
 | 
				
			||||
func (set *Set) Close() error { | 
				
			||||
	if !set.closeUnderlyingDB { | 
				
			||||
		set.db = nil | 
				
			||||
		return nil | 
				
			||||
	} | 
				
			||||
	err := set.db.Close() | 
				
			||||
	set.db = nil | 
				
			||||
	return err | 
				
			||||
} | 
				
			||||
@ -0,0 +1,184 @@ | 
				
			||||
// Copyright 2020 Andrew Thornton. All rights reserved.
 | 
				
			||||
// Use of this source code is governed by a MIT-style
 | 
				
			||||
// license that can be found in the LICENSE file.
 | 
				
			||||
 | 
				
			||||
package levelqueue | 
				
			||||
 | 
				
			||||
import ( | 
				
			||||
	"fmt" | 
				
			||||
 | 
				
			||||
	"github.com/syndtr/goleveldb/leveldb" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
const ( | 
				
			||||
	uniqueQueuePrefixStr = "unique" | 
				
			||||
) | 
				
			||||
 | 
				
			||||
// UniqueQueue defines an unique queue struct
 | 
				
			||||
type UniqueQueue struct { | 
				
			||||
	q                 *Queue | 
				
			||||
	set               *Set | 
				
			||||
	db                *leveldb.DB | 
				
			||||
	closeUnderlyingDB bool | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// OpenUnique opens an unique queue from the db path or creates a set if it doesn't exist.
 | 
				
			||||
// The keys in the queue portion will not be prefixed, and the set keys will be prefixed with "set-"
 | 
				
			||||
func OpenUnique(dataDir string) (*UniqueQueue, error) { | 
				
			||||
	db, err := leveldb.OpenFile(dataDir, nil) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	return NewUniqueQueue(db, []byte{}, []byte(uniqueQueuePrefixStr), true) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// NewUniqueQueue creates a new unique queue from a db.
 | 
				
			||||
// The queue keys will be prefixed with queuePrefix and the set keys with setPrefix
 | 
				
			||||
// and at close the db will be closed as per closeUnderlyingDB
 | 
				
			||||
func NewUniqueQueue(db *leveldb.DB, queuePrefix []byte, setPrefix []byte, closeUnderlyingDB bool) (*UniqueQueue, error) { | 
				
			||||
	internal, err := NewQueue(db, queuePrefix, false) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	set, err := NewSet(db, setPrefix, false) | 
				
			||||
	if err != nil { | 
				
			||||
		return nil, err | 
				
			||||
	} | 
				
			||||
	queue := &UniqueQueue{ | 
				
			||||
		q:                 internal, | 
				
			||||
		set:               set, | 
				
			||||
		db:                db, | 
				
			||||
		closeUnderlyingDB: closeUnderlyingDB, | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	return queue, err | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// LPush pushes data to the left of the queue
 | 
				
			||||
func (queue *UniqueQueue) LPush(data []byte) error { | 
				
			||||
	return queue.LPushFunc(data, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// LPushFunc pushes data to the left of the queue and calls the callback if it is added
 | 
				
			||||
func (queue *UniqueQueue) LPushFunc(data []byte, fn func() error) error { | 
				
			||||
	added, err := queue.set.Add(data) | 
				
			||||
	if err != nil { | 
				
			||||
		return err | 
				
			||||
	} | 
				
			||||
	if !added { | 
				
			||||
		return ErrAlreadyInQueue | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	if fn != nil { | 
				
			||||
		err = fn() | 
				
			||||
		if err != nil { | 
				
			||||
			_, remErr := queue.set.Remove(data) | 
				
			||||
			if remErr != nil { | 
				
			||||
				return fmt.Errorf("%v & %v", err, remErr) | 
				
			||||
			} | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	return queue.q.LPush(data) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// RPush pushes data to the right of the queue
 | 
				
			||||
func (queue *UniqueQueue) RPush(data []byte) error { | 
				
			||||
	return queue.RPushFunc(data, nil) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// RPushFunc pushes data to the right of the queue and calls the callback if is added
 | 
				
			||||
func (queue *UniqueQueue) RPushFunc(data []byte, fn func() error) error { | 
				
			||||
	added, err := queue.set.Add(data) | 
				
			||||
	if err != nil { | 
				
			||||
		return err | 
				
			||||
	} | 
				
			||||
	if !added { | 
				
			||||
		return ErrAlreadyInQueue | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	if fn != nil { | 
				
			||||
		err = fn() | 
				
			||||
		if err != nil { | 
				
			||||
			_, remErr := queue.set.Remove(data) | 
				
			||||
			if remErr != nil { | 
				
			||||
				return fmt.Errorf("%v & %v", err, remErr) | 
				
			||||
			} | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
	} | 
				
			||||
 | 
				
			||||
	return queue.q.RPush(data) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// RPop pop data from the right of the queue
 | 
				
			||||
func (queue *UniqueQueue) RPop() ([]byte, error) { | 
				
			||||
	popped, err := queue.q.RPop() | 
				
			||||
	if err != nil { | 
				
			||||
		return popped, err | 
				
			||||
	} | 
				
			||||
	_, err = queue.set.Remove(popped) | 
				
			||||
 | 
				
			||||
	return popped, err | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// RHandle receives a user callback function to handle the right element of the queue, if the function returns nil, then delete the element, otherwise keep the element.
 | 
				
			||||
func (queue *UniqueQueue) RHandle(h func([]byte) error) error { | 
				
			||||
	return queue.q.RHandle(func(data []byte) error { | 
				
			||||
		err := h(data) | 
				
			||||
		if err != nil { | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
		_, err = queue.set.Remove(data) | 
				
			||||
		return err | 
				
			||||
	}) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// LPop pops data from left of the queue
 | 
				
			||||
func (queue *UniqueQueue) LPop() ([]byte, error) { | 
				
			||||
	popped, err := queue.q.LPop() | 
				
			||||
	if err != nil { | 
				
			||||
		return popped, err | 
				
			||||
	} | 
				
			||||
	_, err = queue.set.Remove(popped) | 
				
			||||
 | 
				
			||||
	return popped, err | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// LHandle receives a user callback function to handle the left element of the queue, if the function returns nil, then delete the element, otherwise keep the element.
 | 
				
			||||
func (queue *UniqueQueue) LHandle(h func([]byte) error) error { | 
				
			||||
	return queue.q.LHandle(func(data []byte) error { | 
				
			||||
		err := h(data) | 
				
			||||
		if err != nil { | 
				
			||||
			return err | 
				
			||||
		} | 
				
			||||
		_, err = queue.set.Remove(data) | 
				
			||||
		return err | 
				
			||||
	}) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Has checks whether the data is already in the queue
 | 
				
			||||
func (queue *UniqueQueue) Has(data []byte) (bool, error) { | 
				
			||||
	return queue.set.Has(data) | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Len returns the length of the queue
 | 
				
			||||
func (queue *UniqueQueue) Len() int64 { | 
				
			||||
	queue.set.lock.Lock() | 
				
			||||
	defer queue.set.lock.Unlock() | 
				
			||||
	return queue.q.Len() | 
				
			||||
} | 
				
			||||
 | 
				
			||||
// Close closes the queue (and the underlying DB if set to closeUnderlyingDB)
 | 
				
			||||
func (queue *UniqueQueue) Close() error { | 
				
			||||
	_ = queue.q.Close() | 
				
			||||
	_ = queue.set.Close() | 
				
			||||
	if !queue.closeUnderlyingDB { | 
				
			||||
		queue.db = nil | 
				
			||||
		return nil | 
				
			||||
	} | 
				
			||||
	err := queue.db.Close() | 
				
			||||
	queue.db = nil | 
				
			||||
	return err | 
				
			||||
} | 
				
			||||
					Loading…
					
					
				
		Reference in new issue