mirror of
				https://github.com/go-gitea/gitea
				synced 2025-10-31 19:38:23 +00:00 
			
		
		
		
	Graceful Queues: Issue Indexing and Tasks (#9363)
* Queue: Add generic graceful queues with settings
* Queue & Setting: Add worker pool implementation
* Queue: Add worker settings
* Queue: Make resizing worker pools
* Queue: Add name variable to queues
* Queue: Add monitoring
* Queue: Improve logging
* Issues: Gracefulise the issues indexer
Remove the old now unused specific queues
* Task: Move to generic queue and gracefulise
* Issues: Standardise the issues indexer queue settings
* Fix test
* Queue: Allow Redis to connect to unix
* Prevent deadlock during early shutdown of issue indexer
* Add MaxWorker settings to queues
* Merge branch 'master' into graceful-queues
* Update modules/indexer/issues/indexer.go
Co-Authored-By: guillep2k <18600385+guillep2k@users.noreply.github.com>
* Update modules/indexer/issues/indexer.go
Co-Authored-By: guillep2k <18600385+guillep2k@users.noreply.github.com>
* Update modules/queue/queue_channel.go
Co-Authored-By: guillep2k <18600385+guillep2k@users.noreply.github.com>
* Update modules/queue/queue_disk.go
* Update modules/queue/queue_disk_channel.go
Co-Authored-By: guillep2k <18600385+guillep2k@users.noreply.github.com>
* Rename queue.Description to queue.ManagedQueue as per @guillep2k
* Cancel pool workers when removed
* Remove dependency on queue from setting
* Update modules/queue/queue_redis.go
Co-Authored-By: guillep2k <18600385+guillep2k@users.noreply.github.com>
* As per @guillep2k add mutex locks on shutdown/terminate
* move unlocking out of setInternal
* Add warning if number of workers < 0
* Small changes as per @guillep2k
* No redis host specified not found
* Clean up documentation for queues
* Update docs/content/doc/advanced/config-cheat-sheet.en-us.md
* Update modules/indexer/issues/indexer_test.go
* Ensure that persistable channel queue is added to manager
* Rename QUEUE_NAME REDIS_QUEUE_NAME
* Revert "Rename QUEUE_NAME REDIS_QUEUE_NAME"
This reverts commit 1f83b4fc9b.
Co-authored-by: guillep2k <18600385+guillep2k@users.noreply.github.com>
Co-authored-by: Lauris BH <lauris@nix.lv>
Co-authored-by: techknowlogick <matti@mdranta.net>
Co-authored-by: Lunny Xiao <xiaolunwen@gmail.com>
			
			
This commit is contained in:
		| @@ -382,6 +382,39 @@ REPO_INDEXER_INCLUDE = | |||||||
| ; A comma separated list of glob patterns to exclude from the index; ; default is empty | ; A comma separated list of glob patterns to exclude from the index; ; default is empty | ||||||
| REPO_INDEXER_EXCLUDE = | REPO_INDEXER_EXCLUDE = | ||||||
|  |  | ||||||
|  | [queue] | ||||||
|  | ; Specific queues can be individually configured with [queue.name]. [queue] provides defaults | ||||||
|  | ; | ||||||
|  | ; General queue queue type, currently support: persistable-channel, channel, level, redis, dummy | ||||||
|  | ; default to persistable-channel | ||||||
|  | TYPE = persistable-channel | ||||||
|  | ; data-dir for storing persistable queues and level queues, individual queues will be named by their type | ||||||
|  | DATADIR = queues/ | ||||||
|  | ; Default queue length before a channel queue will block | ||||||
|  | LENGTH = 20 | ||||||
|  | ; Batch size to send for batched queues | ||||||
|  | BATCH_LENGTH = 20 | ||||||
|  | ; Connection string for redis queues this will store the redis connection string. | ||||||
|  | CONN_STR = "addrs=127.0.0.1:6379 db=0" | ||||||
|  | ; Provide the suffix of the default redis queue name - specific queues can be overriden within in their [queue.name] sections. | ||||||
|  | QUEUE_NAME = "_queue" | ||||||
|  | ; If the queue cannot be created at startup - level queues may need a timeout at startup - wrap the queue: | ||||||
|  | WRAP_IF_NECESSARY = true | ||||||
|  | ; Attempt to create the wrapped queue at max | ||||||
|  | MAX_ATTEMPTS = 10 | ||||||
|  | ; Timeout queue creation | ||||||
|  | TIMEOUT = 15m30s | ||||||
|  | ; Create a pool with this many workers | ||||||
|  | WORKERS = 1 | ||||||
|  | ; Dynamically scale the worker pool to at this many workers | ||||||
|  | MAX_WORKERS = 10 | ||||||
|  | ; Add boost workers when the queue blocks for BLOCK_TIMEOUT | ||||||
|  | BLOCK_TIMEOUT = 1s | ||||||
|  | ; Remove the boost workers after BOOST_TIMEOUT | ||||||
|  | BOOST_TIMEOUT = 5m | ||||||
|  | ; During a boost add BOOST_WORKERS | ||||||
|  | BOOST_WORKERS = 5 | ||||||
|  |  | ||||||
| [admin] | [admin] | ||||||
| ; Disallow regular (non-admin) users from creating organizations. | ; Disallow regular (non-admin) users from creating organizations. | ||||||
| DISABLE_REGULAR_ORG_CREATION = false | DISABLE_REGULAR_ORG_CREATION = false | ||||||
|   | |||||||
| @@ -226,6 +226,7 @@ relation to port exhaustion. | |||||||
|  |  | ||||||
| - `ISSUE_INDEXER_TYPE`: **bleve**: Issue indexer type, currently support: bleve or db, if it's db, below issue indexer item will be invalid. | - `ISSUE_INDEXER_TYPE`: **bleve**: Issue indexer type, currently support: bleve or db, if it's db, below issue indexer item will be invalid. | ||||||
| - `ISSUE_INDEXER_PATH`: **indexers/issues.bleve**: Index file used for issue search. | - `ISSUE_INDEXER_PATH`: **indexers/issues.bleve**: Index file used for issue search. | ||||||
|  | - The next 4 configuration values are deprecated and should be set in `queue.issue_indexer` however are kept for backwards compatibility: | ||||||
| - `ISSUE_INDEXER_QUEUE_TYPE`: **levelqueue**: Issue indexer queue, currently supports:`channel`, `levelqueue`, `redis`. | - `ISSUE_INDEXER_QUEUE_TYPE`: **levelqueue**: Issue indexer queue, currently supports:`channel`, `levelqueue`, `redis`. | ||||||
| - `ISSUE_INDEXER_QUEUE_DIR`: **indexers/issues.queue**: When `ISSUE_INDEXER_QUEUE_TYPE` is `levelqueue`, this will be the queue will be saved path. | - `ISSUE_INDEXER_QUEUE_DIR`: **indexers/issues.queue**: When `ISSUE_INDEXER_QUEUE_TYPE` is `levelqueue`, this will be the queue will be saved path. | ||||||
| - `ISSUE_INDEXER_QUEUE_CONN_STR`: **addrs=127.0.0.1:6379 db=0**: When `ISSUE_INDEXER_QUEUE_TYPE` is `redis`, this will store the redis connection string. | - `ISSUE_INDEXER_QUEUE_CONN_STR`: **addrs=127.0.0.1:6379 db=0**: When `ISSUE_INDEXER_QUEUE_TYPE` is `redis`, this will store the redis connection string. | ||||||
| @@ -239,6 +240,24 @@ relation to port exhaustion. | |||||||
| - `MAX_FILE_SIZE`: **1048576**: Maximum size in bytes of files to be indexed. | - `MAX_FILE_SIZE`: **1048576**: Maximum size in bytes of files to be indexed. | ||||||
| - `STARTUP_TIMEOUT`: **30s**: If the indexer takes longer than this timeout to start - fail. (This timeout will be added to the hammer time above for child processes - as bleve will not start until the previous parent is shutdown.) Set to zero to never timeout. | - `STARTUP_TIMEOUT`: **30s**: If the indexer takes longer than this timeout to start - fail. (This timeout will be added to the hammer time above for child processes - as bleve will not start until the previous parent is shutdown.) Set to zero to never timeout. | ||||||
|  |  | ||||||
|  | ## Queue (`queue` and `queue.*`) | ||||||
|  |  | ||||||
|  | - `TYPE`: **persistable-channel**: General queue type, currently support: `persistable-channel`, `channel`, `level`, `redis`, `dummy` | ||||||
|  | - `DATADIR`: **queues/**: Base DataDir for storing persistent and level queues. `DATADIR` for inidividual queues can be set in `queue.name` sections but will default to `DATADIR/`**`name`**. | ||||||
|  | - `LENGTH`: **20**: Maximal queue size before channel queues block | ||||||
|  | - `BATCH_LENGTH`: **20**: Batch data before passing to the handler | ||||||
|  | - `CONN_STR`: **addrs=127.0.0.1:6379 db=0**: Connection string for the redis queue type. | ||||||
|  | - `QUEUE_NAME`: **_queue**: The suffix for default redis queue name. Individual queues will default to **`name`**`QUEUE_NAME` but can be overriden in the specific `queue.name` section. | ||||||
|  | - `WRAP_IF_NECESSARY`: **true**: Will wrap queues with a timeoutable queue if the selected queue is not ready to be created - (Only relevant for the level queue.) | ||||||
|  | - `MAX_ATTEMPTS`: **10**: Maximum number of attempts to create the wrapped queue | ||||||
|  | - `TIMEOUT`: **GRACEFUL_HAMMER_TIME + 30s**: Timeout the creation of the wrapped queue if it takes longer than this to create. | ||||||
|  | - Queues by default come with a dynamically scaling worker pool. The following settings configure this: | ||||||
|  | - `WORKERS`: **1**: Number of initial workers for the queue. | ||||||
|  | - `MAX_WORKERS`: **10**: Maximum number of worker go-routines for the queue. | ||||||
|  | - `BLOCK_TIMEOUT`: **1s**: If the queue blocks for this time, boost the number of workers - the `BLOCK_TIMEOUT` will then be doubled before boosting again whilst the boost is ongoing. | ||||||
|  | - `BOOST_TIMEOUT`: **5m**: Boost workers will timeout after this long. | ||||||
|  | - `BOOST_WORKERS`: **5**: This many workers will be added to the worker pool if there is a boost. | ||||||
|  |  | ||||||
| ## Admin (`admin`) | ## Admin (`admin`) | ||||||
| - `DEFAULT_EMAIL_NOTIFICATIONS`: **enabled**: Default configuration for email notifications for users (user configurable). Options: enabled, onmention, disabled | - `DEFAULT_EMAIL_NOTIFICATIONS`: **enabled**: Default configuration for email notifications for users (user configurable). Options: enabled, onmention, disabled | ||||||
|  |  | ||||||
| @@ -614,6 +633,7 @@ You may redefine `ELEMENT`, `ALLOW_ATTR`, and `REGEXP` multiple times; each time | |||||||
|  |  | ||||||
| ## Task (`task`) | ## Task (`task`) | ||||||
|  |  | ||||||
|  | -  Task queue configuration has been moved to `queue.task` however, the below configuration values are kept for backwards compatibilityx: | ||||||
| - `QUEUE_TYPE`: **channel**: Task queue type, could be `channel` or `redis`. | - `QUEUE_TYPE`: **channel**: Task queue type, could be `channel` or `redis`. | ||||||
| - `QUEUE_LENGTH`: **1000**: Task queue length, available only when `QUEUE_TYPE` is `channel`. | - `QUEUE_LENGTH`: **1000**: Task queue length, available only when `QUEUE_TYPE` is `channel`. | ||||||
| - `QUEUE_CONN_STR`: **addrs=127.0.0.1:6379 db=0**: Task queue connection string, available only when `QUEUE_TYPE` is `redis`. If there redis needs a password, use `addrs=127.0.0.1:6379 password=123 db=0`. | - `QUEUE_CONN_STR`: **addrs=127.0.0.1:6379 db=0**: Task queue connection string, available only when `QUEUE_TYPE` is `redis`. If there redis needs a password, use `addrs=127.0.0.1:6379 password=123 db=0`. | ||||||
|   | |||||||
| @@ -11,8 +11,10 @@ import ( | |||||||
| 	"strconv" | 	"strconv" | ||||||
| 	"strings" | 	"strings" | ||||||
| 	"testing" | 	"testing" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/models" | 	"code.gitea.io/gitea/models" | ||||||
|  | 	"code.gitea.io/gitea/modules/indexer/issues" | ||||||
| 	"code.gitea.io/gitea/modules/references" | 	"code.gitea.io/gitea/modules/references" | ||||||
| 	"code.gitea.io/gitea/modules/setting" | 	"code.gitea.io/gitea/modules/setting" | ||||||
| 	"code.gitea.io/gitea/modules/test" | 	"code.gitea.io/gitea/modules/test" | ||||||
| @@ -87,7 +89,12 @@ func TestViewIssuesKeyword(t *testing.T) { | |||||||
| 	defer prepareTestEnv(t)() | 	defer prepareTestEnv(t)() | ||||||
|  |  | ||||||
| 	repo := models.AssertExistsAndLoadBean(t, &models.Repository{ID: 1}).(*models.Repository) | 	repo := models.AssertExistsAndLoadBean(t, &models.Repository{ID: 1}).(*models.Repository) | ||||||
|  | 	issue := models.AssertExistsAndLoadBean(t, &models.Issue{ | ||||||
|  | 		RepoID: repo.ID, | ||||||
|  | 		Index:  1, | ||||||
|  | 	}).(*models.Issue) | ||||||
|  | 	issues.UpdateIssueIndexer(issue) | ||||||
|  | 	time.Sleep(time.Second * 1) | ||||||
| 	const keyword = "first" | 	const keyword = "first" | ||||||
| 	req := NewRequestf(t, "GET", "%s/issues?q=%s", repo.RelLink(), keyword) | 	req := NewRequestf(t, "GET", "%s/issues?q=%s", repo.RelLink(), keyword) | ||||||
| 	resp := MakeRequest(t, req, http.StatusOK) | 	resp := MakeRequest(t, req, http.StatusOK) | ||||||
|   | |||||||
| @@ -25,6 +25,10 @@ func (db *DBIndexer) Delete(ids ...int64) error { | |||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // Close dummy function | ||||||
|  | func (db *DBIndexer) Close() { | ||||||
|  | } | ||||||
|  |  | ||||||
| // Search dummy function | // Search dummy function | ||||||
| func (db *DBIndexer) Search(kw string, repoIDs []int64, limit, start int) (*SearchResult, error) { | func (db *DBIndexer) Search(kw string, repoIDs []int64, limit, start int) (*SearchResult, error) { | ||||||
| 	total, ids, err := models.SearchIssueIDsByKeyword(kw, repoIDs, limit, start) | 	total, ids, err := models.SearchIssueIDsByKeyword(kw, repoIDs, limit, start) | ||||||
|   | |||||||
| @@ -5,12 +5,16 @@ | |||||||
| package issues | package issues | ||||||
|  |  | ||||||
| import ( | import ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 	"os" | ||||||
| 	"sync" | 	"sync" | ||||||
| 	"time" | 	"time" | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/models" | 	"code.gitea.io/gitea/models" | ||||||
| 	"code.gitea.io/gitea/modules/graceful" | 	"code.gitea.io/gitea/modules/graceful" | ||||||
| 	"code.gitea.io/gitea/modules/log" | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | 	"code.gitea.io/gitea/modules/queue" | ||||||
| 	"code.gitea.io/gitea/modules/setting" | 	"code.gitea.io/gitea/modules/setting" | ||||||
| 	"code.gitea.io/gitea/modules/util" | 	"code.gitea.io/gitea/modules/util" | ||||||
| ) | ) | ||||||
| @@ -44,12 +48,14 @@ type Indexer interface { | |||||||
| 	Index(issue []*IndexerData) error | 	Index(issue []*IndexerData) error | ||||||
| 	Delete(ids ...int64) error | 	Delete(ids ...int64) error | ||||||
| 	Search(kw string, repoIDs []int64, limit, start int) (*SearchResult, error) | 	Search(kw string, repoIDs []int64, limit, start int) (*SearchResult, error) | ||||||
|  | 	Close() | ||||||
| } | } | ||||||
|  |  | ||||||
| type indexerHolder struct { | type indexerHolder struct { | ||||||
| 	indexer   Indexer | 	indexer   Indexer | ||||||
| 	mutex     sync.RWMutex | 	mutex     sync.RWMutex | ||||||
| 	cond      *sync.Cond | 	cond      *sync.Cond | ||||||
|  | 	cancelled bool | ||||||
| } | } | ||||||
|  |  | ||||||
| func newIndexerHolder() *indexerHolder { | func newIndexerHolder() *indexerHolder { | ||||||
| @@ -58,6 +64,13 @@ func newIndexerHolder() *indexerHolder { | |||||||
| 	return h | 	return h | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func (h *indexerHolder) cancel() { | ||||||
|  | 	h.mutex.Lock() | ||||||
|  | 	defer h.mutex.Unlock() | ||||||
|  | 	h.cancelled = true | ||||||
|  | 	h.cond.Broadcast() | ||||||
|  | } | ||||||
|  |  | ||||||
| func (h *indexerHolder) set(indexer Indexer) { | func (h *indexerHolder) set(indexer Indexer) { | ||||||
| 	h.mutex.Lock() | 	h.mutex.Lock() | ||||||
| 	defer h.mutex.Unlock() | 	defer h.mutex.Unlock() | ||||||
| @@ -68,16 +81,15 @@ func (h *indexerHolder) set(indexer Indexer) { | |||||||
| func (h *indexerHolder) get() Indexer { | func (h *indexerHolder) get() Indexer { | ||||||
| 	h.mutex.RLock() | 	h.mutex.RLock() | ||||||
| 	defer h.mutex.RUnlock() | 	defer h.mutex.RUnlock() | ||||||
| 	if h.indexer == nil { | 	if h.indexer == nil && !h.cancelled { | ||||||
| 		h.cond.Wait() | 		h.cond.Wait() | ||||||
| 	} | 	} | ||||||
| 	return h.indexer | 	return h.indexer | ||||||
| } | } | ||||||
|  |  | ||||||
| var ( | var ( | ||||||
| 	issueIndexerChannel = make(chan *IndexerData, setting.Indexer.UpdateQueueLength) |  | ||||||
| 	// issueIndexerQueue queue of issue ids to be updated | 	// issueIndexerQueue queue of issue ids to be updated | ||||||
| 	issueIndexerQueue Queue | 	issueIndexerQueue queue.Queue | ||||||
| 	holder            = newIndexerHolder() | 	holder            = newIndexerHolder() | ||||||
| ) | ) | ||||||
|  |  | ||||||
| @@ -85,90 +97,99 @@ var ( | |||||||
| // all issue index done. | // all issue index done. | ||||||
| func InitIssueIndexer(syncReindex bool) { | func InitIssueIndexer(syncReindex bool) { | ||||||
| 	waitChannel := make(chan time.Duration) | 	waitChannel := make(chan time.Duration) | ||||||
| 	go func() { |  | ||||||
| 		start := time.Now() | 	// Create the Queue | ||||||
| 		log.Info("Initializing Issue Indexer") |  | ||||||
| 		var populate bool |  | ||||||
| 		var dummyQueue bool |  | ||||||
| 	switch setting.Indexer.IssueType { | 	switch setting.Indexer.IssueType { | ||||||
| 	case "bleve": | 	case "bleve": | ||||||
|  | 		handler := func(data ...queue.Data) { | ||||||
|  | 			indexer := holder.get() | ||||||
|  | 			if indexer == nil { | ||||||
|  | 				log.Error("Issue indexer handler: unable to get indexer!") | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			iData := make([]*IndexerData, 0, setting.Indexer.IssueQueueBatchNumber) | ||||||
|  | 			for _, datum := range data { | ||||||
|  | 				indexerData, ok := datum.(*IndexerData) | ||||||
|  | 				if !ok { | ||||||
|  | 					log.Error("Unable to process provided datum: %v - not possible to cast to IndexerData", datum) | ||||||
|  | 					continue | ||||||
|  | 				} | ||||||
|  | 				log.Trace("IndexerData Process: %d %v %t", indexerData.ID, indexerData.IDs, indexerData.IsDelete) | ||||||
|  | 				if indexerData.IsDelete { | ||||||
|  | 					_ = indexer.Delete(indexerData.IDs...) | ||||||
|  | 					continue | ||||||
|  | 				} | ||||||
|  | 				iData = append(iData, indexerData) | ||||||
|  | 			} | ||||||
|  | 			if err := indexer.Index(iData); err != nil { | ||||||
|  | 				log.Error("Error whilst indexing: %v Error: %v", iData, err) | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 		issueIndexerQueue = queue.CreateQueue("issue_indexer", handler, &IndexerData{}) | ||||||
|  |  | ||||||
|  | 		if issueIndexerQueue == nil { | ||||||
|  | 			log.Fatal("Unable to create issue indexer queue") | ||||||
|  | 		} | ||||||
|  | 	default: | ||||||
|  | 		issueIndexerQueue = &queue.DummyQueue{} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// Create the Indexer | ||||||
|  | 	go func() { | ||||||
|  | 		start := time.Now() | ||||||
|  | 		log.Info("PID %d: Initializing Issue Indexer: %s", os.Getpid(), setting.Indexer.IssueType) | ||||||
|  | 		var populate bool | ||||||
|  | 		switch setting.Indexer.IssueType { | ||||||
|  | 		case "bleve": | ||||||
|  | 			graceful.GetManager().RunWithShutdownFns(func(_, atTerminate func(context.Context, func())) { | ||||||
| 				issueIndexer := NewBleveIndexer(setting.Indexer.IssuePath) | 				issueIndexer := NewBleveIndexer(setting.Indexer.IssuePath) | ||||||
| 				exist, err := issueIndexer.Init() | 				exist, err := issueIndexer.Init() | ||||||
| 				if err != nil { | 				if err != nil { | ||||||
|  | 					holder.cancel() | ||||||
| 					log.Fatal("Unable to initialize Bleve Issue Indexer: %v", err) | 					log.Fatal("Unable to initialize Bleve Issue Indexer: %v", err) | ||||||
| 				} | 				} | ||||||
| 				populate = !exist | 				populate = !exist | ||||||
| 				holder.set(issueIndexer) | 				holder.set(issueIndexer) | ||||||
|  | 				atTerminate(context.Background(), func() { | ||||||
|  | 					log.Debug("Closing issue indexer") | ||||||
|  | 					issueIndexer := holder.get() | ||||||
|  | 					if issueIndexer != nil { | ||||||
|  | 						issueIndexer.Close() | ||||||
|  | 					} | ||||||
|  | 					log.Info("PID: %d Issue Indexer closed", os.Getpid()) | ||||||
|  | 				}) | ||||||
|  | 				log.Debug("Created Bleve Indexer") | ||||||
|  | 			}) | ||||||
| 		case "db": | 		case "db": | ||||||
| 			issueIndexer := &DBIndexer{} | 			issueIndexer := &DBIndexer{} | ||||||
| 			holder.set(issueIndexer) | 			holder.set(issueIndexer) | ||||||
| 			dummyQueue = true |  | ||||||
| 		default: | 		default: | ||||||
|  | 			holder.cancel() | ||||||
| 			log.Fatal("Unknown issue indexer type: %s", setting.Indexer.IssueType) | 			log.Fatal("Unknown issue indexer type: %s", setting.Indexer.IssueType) | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
| 		if dummyQueue { | 		// Start processing the queue | ||||||
| 			issueIndexerQueue = &DummyQueue{} | 		go graceful.GetManager().RunWithShutdownFns(issueIndexerQueue.Run) | ||||||
| 		} else { |  | ||||||
| 			var err error |  | ||||||
| 			switch setting.Indexer.IssueQueueType { |  | ||||||
| 			case setting.LevelQueueType: |  | ||||||
| 				issueIndexerQueue, err = NewLevelQueue( |  | ||||||
| 					holder.get(), |  | ||||||
| 					setting.Indexer.IssueQueueDir, |  | ||||||
| 					setting.Indexer.IssueQueueBatchNumber) |  | ||||||
| 				if err != nil { |  | ||||||
| 					log.Fatal( |  | ||||||
| 						"Unable create level queue for issue queue dir: %s batch number: %d : %v", |  | ||||||
| 						setting.Indexer.IssueQueueDir, |  | ||||||
| 						setting.Indexer.IssueQueueBatchNumber, |  | ||||||
| 						err) |  | ||||||
| 				} |  | ||||||
| 			case setting.ChannelQueueType: |  | ||||||
| 				issueIndexerQueue = NewChannelQueue(holder.get(), setting.Indexer.IssueQueueBatchNumber) |  | ||||||
| 			case setting.RedisQueueType: |  | ||||||
| 				addrs, pass, idx, err := parseConnStr(setting.Indexer.IssueQueueConnStr) |  | ||||||
| 				if err != nil { |  | ||||||
| 					log.Fatal("Unable to parse connection string for RedisQueueType: %s : %v", |  | ||||||
| 						setting.Indexer.IssueQueueConnStr, |  | ||||||
| 						err) |  | ||||||
| 				} |  | ||||||
| 				issueIndexerQueue, err = NewRedisQueue(addrs, pass, idx, holder.get(), setting.Indexer.IssueQueueBatchNumber) |  | ||||||
| 				if err != nil { |  | ||||||
| 					log.Fatal("Unable to create RedisQueue: %s : %v", |  | ||||||
| 						setting.Indexer.IssueQueueConnStr, |  | ||||||
| 						err) |  | ||||||
| 				} |  | ||||||
| 			default: |  | ||||||
| 				log.Fatal("Unsupported indexer queue type: %v", |  | ||||||
| 					setting.Indexer.IssueQueueType) |  | ||||||
| 			} |  | ||||||
|  |  | ||||||
| 			go func() { |  | ||||||
| 				err = issueIndexerQueue.Run() |  | ||||||
| 				if err != nil { |  | ||||||
| 					log.Error("issueIndexerQueue.Run: %v", err) |  | ||||||
| 				} |  | ||||||
| 			}() |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		go func() { |  | ||||||
| 			for data := range issueIndexerChannel { |  | ||||||
| 				_ = issueIndexerQueue.Push(data) |  | ||||||
| 			} |  | ||||||
| 		}() |  | ||||||
|  |  | ||||||
|  | 		// Populate the index | ||||||
| 		if populate { | 		if populate { | ||||||
| 			if syncReindex { | 			if syncReindex { | ||||||
| 				populateIssueIndexer() | 				graceful.GetManager().RunWithShutdownContext(populateIssueIndexer) | ||||||
| 			} else { | 			} else { | ||||||
| 				go populateIssueIndexer() | 				go graceful.GetManager().RunWithShutdownContext(populateIssueIndexer) | ||||||
| 			} | 			} | ||||||
| 		} | 		} | ||||||
| 		waitChannel <- time.Since(start) | 		waitChannel <- time.Since(start) | ||||||
|  | 		close(waitChannel) | ||||||
| 	}() | 	}() | ||||||
|  |  | ||||||
| 	if syncReindex { | 	if syncReindex { | ||||||
| 		<-waitChannel | 		select { | ||||||
|  | 		case <-waitChannel: | ||||||
|  | 		case <-graceful.GetManager().IsShutdown(): | ||||||
|  | 		} | ||||||
| 	} else if setting.Indexer.StartupTimeout > 0 { | 	} else if setting.Indexer.StartupTimeout > 0 { | ||||||
| 		go func() { | 		go func() { | ||||||
| 			timeout := setting.Indexer.StartupTimeout | 			timeout := setting.Indexer.StartupTimeout | ||||||
| @@ -178,7 +199,12 @@ func InitIssueIndexer(syncReindex bool) { | |||||||
| 			select { | 			select { | ||||||
| 			case duration := <-waitChannel: | 			case duration := <-waitChannel: | ||||||
| 				log.Info("Issue Indexer Initialization took %v", duration) | 				log.Info("Issue Indexer Initialization took %v", duration) | ||||||
|  | 			case <-graceful.GetManager().IsShutdown(): | ||||||
|  | 				log.Warn("Shutdown occurred before issue index initialisation was complete") | ||||||
| 			case <-time.After(timeout): | 			case <-time.After(timeout): | ||||||
|  | 				if shutdownable, ok := issueIndexerQueue.(queue.Shutdownable); ok { | ||||||
|  | 					shutdownable.Terminate() | ||||||
|  | 				} | ||||||
| 				log.Fatal("Issue Indexer Initialization timed-out after: %v", timeout) | 				log.Fatal("Issue Indexer Initialization timed-out after: %v", timeout) | ||||||
| 			} | 			} | ||||||
| 		}() | 		}() | ||||||
| @@ -186,8 +212,14 @@ func InitIssueIndexer(syncReindex bool) { | |||||||
| } | } | ||||||
|  |  | ||||||
| // populateIssueIndexer populate the issue indexer with issue data | // populateIssueIndexer populate the issue indexer with issue data | ||||||
| func populateIssueIndexer() { | func populateIssueIndexer(ctx context.Context) { | ||||||
| 	for page := 1; ; page++ { | 	for page := 1; ; page++ { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			log.Warn("Issue Indexer population shutdown before completion") | ||||||
|  | 			return | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
| 		repos, _, err := models.SearchRepositoryByName(&models.SearchRepoOptions{ | 		repos, _, err := models.SearchRepositoryByName(&models.SearchRepoOptions{ | ||||||
| 			Page:        page, | 			Page:        page, | ||||||
| 			PageSize:    models.RepositoryListDefaultPageSize, | 			PageSize:    models.RepositoryListDefaultPageSize, | ||||||
| @@ -200,10 +232,17 @@ func populateIssueIndexer() { | |||||||
| 			continue | 			continue | ||||||
| 		} | 		} | ||||||
| 		if len(repos) == 0 { | 		if len(repos) == 0 { | ||||||
|  | 			log.Debug("Issue Indexer population complete") | ||||||
| 			return | 			return | ||||||
| 		} | 		} | ||||||
|  |  | ||||||
| 		for _, repo := range repos { | 		for _, repo := range repos { | ||||||
|  | 			select { | ||||||
|  | 			case <-ctx.Done(): | ||||||
|  | 				log.Info("Issue Indexer population shutdown before completion") | ||||||
|  | 				return | ||||||
|  | 			default: | ||||||
|  | 			} | ||||||
| 			UpdateRepoIndexer(repo) | 			UpdateRepoIndexer(repo) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| @@ -237,13 +276,17 @@ func UpdateIssueIndexer(issue *models.Issue) { | |||||||
| 			comments = append(comments, comment.Content) | 			comments = append(comments, comment.Content) | ||||||
| 		} | 		} | ||||||
| 	} | 	} | ||||||
| 	issueIndexerChannel <- &IndexerData{ | 	indexerData := &IndexerData{ | ||||||
| 		ID:       issue.ID, | 		ID:       issue.ID, | ||||||
| 		RepoID:   issue.RepoID, | 		RepoID:   issue.RepoID, | ||||||
| 		Title:    issue.Title, | 		Title:    issue.Title, | ||||||
| 		Content:  issue.Content, | 		Content:  issue.Content, | ||||||
| 		Comments: comments, | 		Comments: comments, | ||||||
| 	} | 	} | ||||||
|  | 	log.Debug("Adding to channel: %v", indexerData) | ||||||
|  | 	if err := issueIndexerQueue.Push(indexerData); err != nil { | ||||||
|  | 		log.Error("Unable to push to issue indexer: %v: Error: %v", indexerData, err) | ||||||
|  | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
| // DeleteRepoIssueIndexer deletes repo's all issues indexes | // DeleteRepoIssueIndexer deletes repo's all issues indexes | ||||||
| @@ -258,17 +301,25 @@ func DeleteRepoIssueIndexer(repo *models.Repository) { | |||||||
| 	if len(ids) == 0 { | 	if len(ids) == 0 { | ||||||
| 		return | 		return | ||||||
| 	} | 	} | ||||||
|  | 	indexerData := &IndexerData{ | ||||||
| 	issueIndexerChannel <- &IndexerData{ |  | ||||||
| 		IDs:      ids, | 		IDs:      ids, | ||||||
| 		IsDelete: true, | 		IsDelete: true, | ||||||
| 	} | 	} | ||||||
|  | 	if err := issueIndexerQueue.Push(indexerData); err != nil { | ||||||
|  | 		log.Error("Unable to push to issue indexer: %v: Error: %v", indexerData, err) | ||||||
|  | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
| // SearchIssuesByKeyword search issue ids by keywords and repo id | // SearchIssuesByKeyword search issue ids by keywords and repo id | ||||||
| func SearchIssuesByKeyword(repoIDs []int64, keyword string) ([]int64, error) { | func SearchIssuesByKeyword(repoIDs []int64, keyword string) ([]int64, error) { | ||||||
| 	var issueIDs []int64 | 	var issueIDs []int64 | ||||||
| 	res, err := holder.get().Search(keyword, repoIDs, 1000, 0) | 	indexer := holder.get() | ||||||
|  |  | ||||||
|  | 	if indexer == nil { | ||||||
|  | 		log.Error("SearchIssuesByKeyword(): unable to get indexer!") | ||||||
|  | 		return nil, fmt.Errorf("unable to get issue indexer") | ||||||
|  | 	} | ||||||
|  | 	res, err := indexer.Search(keyword, repoIDs, 1000, 0) | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| 		return nil, err | 		return nil, err | ||||||
| 	} | 	} | ||||||
|   | |||||||
| @@ -15,6 +15,8 @@ import ( | |||||||
| 	"code.gitea.io/gitea/models" | 	"code.gitea.io/gitea/models" | ||||||
| 	"code.gitea.io/gitea/modules/setting" | 	"code.gitea.io/gitea/modules/setting" | ||||||
|  |  | ||||||
|  | 	"gopkg.in/ini.v1" | ||||||
|  |  | ||||||
| 	"github.com/stretchr/testify/assert" | 	"github.com/stretchr/testify/assert" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| @@ -24,6 +26,7 @@ func TestMain(m *testing.M) { | |||||||
|  |  | ||||||
| func TestBleveSearchIssues(t *testing.T) { | func TestBleveSearchIssues(t *testing.T) { | ||||||
| 	assert.NoError(t, models.PrepareTestDatabase()) | 	assert.NoError(t, models.PrepareTestDatabase()) | ||||||
|  | 	setting.Cfg = ini.Empty() | ||||||
|  |  | ||||||
| 	tmpIndexerDir, err := ioutil.TempDir("", "issues-indexer") | 	tmpIndexerDir, err := ioutil.TempDir("", "issues-indexer") | ||||||
| 	if err != nil { | 	if err != nil { | ||||||
| @@ -41,6 +44,7 @@ func TestBleveSearchIssues(t *testing.T) { | |||||||
| 	}() | 	}() | ||||||
|  |  | ||||||
| 	setting.Indexer.IssueType = "bleve" | 	setting.Indexer.IssueType = "bleve" | ||||||
|  | 	setting.NewQueueService() | ||||||
| 	InitIssueIndexer(true) | 	InitIssueIndexer(true) | ||||||
| 	defer func() { | 	defer func() { | ||||||
| 		indexer := holder.get() | 		indexer := holder.get() | ||||||
|   | |||||||
| @@ -1,25 +0,0 @@ | |||||||
| // Copyright 2018 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 issues |  | ||||||
|  |  | ||||||
| // Queue defines an interface to save an issue indexer queue |  | ||||||
| type Queue interface { |  | ||||||
| 	Run() error |  | ||||||
| 	Push(*IndexerData) error |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // DummyQueue represents an empty queue |  | ||||||
| type DummyQueue struct { |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Run starts to run the queue |  | ||||||
| func (b *DummyQueue) Run() error { |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Push pushes data to indexer |  | ||||||
| func (b *DummyQueue) Push(*IndexerData) error { |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
| @@ -1,62 +0,0 @@ | |||||||
| // Copyright 2018 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 issues |  | ||||||
|  |  | ||||||
| import ( |  | ||||||
| 	"time" |  | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/modules/setting" |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| // ChannelQueue implements |  | ||||||
| type ChannelQueue struct { |  | ||||||
| 	queue       chan *IndexerData |  | ||||||
| 	indexer     Indexer |  | ||||||
| 	batchNumber int |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // NewChannelQueue create a memory channel queue |  | ||||||
| func NewChannelQueue(indexer Indexer, batchNumber int) *ChannelQueue { |  | ||||||
| 	return &ChannelQueue{ |  | ||||||
| 		queue:       make(chan *IndexerData, setting.Indexer.UpdateQueueLength), |  | ||||||
| 		indexer:     indexer, |  | ||||||
| 		batchNumber: batchNumber, |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Run starts to run the queue |  | ||||||
| func (c *ChannelQueue) Run() error { |  | ||||||
| 	var i int |  | ||||||
| 	var datas = make([]*IndexerData, 0, c.batchNumber) |  | ||||||
| 	for { |  | ||||||
| 		select { |  | ||||||
| 		case data := <-c.queue: |  | ||||||
| 			if data.IsDelete { |  | ||||||
| 				_ = c.indexer.Delete(data.IDs...) |  | ||||||
| 				continue |  | ||||||
| 			} |  | ||||||
|  |  | ||||||
| 			datas = append(datas, data) |  | ||||||
| 			if len(datas) >= c.batchNumber { |  | ||||||
| 				_ = c.indexer.Index(datas) |  | ||||||
| 				// TODO: save the point |  | ||||||
| 				datas = make([]*IndexerData, 0, c.batchNumber) |  | ||||||
| 			} |  | ||||||
| 		case <-time.After(time.Millisecond * 100): |  | ||||||
| 			i++ |  | ||||||
| 			if i >= 3 && len(datas) > 0 { |  | ||||||
| 				_ = c.indexer.Index(datas) |  | ||||||
| 				// TODO: save the point |  | ||||||
| 				datas = make([]*IndexerData, 0, c.batchNumber) |  | ||||||
| 			} |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Push will push the indexer data to queue |  | ||||||
| func (c *ChannelQueue) Push(data *IndexerData) error { |  | ||||||
| 	c.queue <- data |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
| @@ -1,104 +0,0 @@ | |||||||
| // 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 issues |  | ||||||
|  |  | ||||||
| import ( |  | ||||||
| 	"encoding/json" |  | ||||||
| 	"time" |  | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/modules/log" |  | ||||||
|  |  | ||||||
| 	"gitea.com/lunny/levelqueue" |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| var ( |  | ||||||
| 	_ Queue = &LevelQueue{} |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| // LevelQueue implements a disk library queue |  | ||||||
| type LevelQueue struct { |  | ||||||
| 	indexer     Indexer |  | ||||||
| 	queue       *levelqueue.Queue |  | ||||||
| 	batchNumber int |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // NewLevelQueue creates a ledis local queue |  | ||||||
| func NewLevelQueue(indexer Indexer, dataDir string, batchNumber int) (*LevelQueue, error) { |  | ||||||
| 	queue, err := levelqueue.Open(dataDir) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return nil, err |  | ||||||
| 	} |  | ||||||
|  |  | ||||||
| 	return &LevelQueue{ |  | ||||||
| 		indexer:     indexer, |  | ||||||
| 		queue:       queue, |  | ||||||
| 		batchNumber: batchNumber, |  | ||||||
| 	}, nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Run starts to run the queue |  | ||||||
| func (l *LevelQueue) Run() error { |  | ||||||
| 	var i int |  | ||||||
| 	var datas = make([]*IndexerData, 0, l.batchNumber) |  | ||||||
| 	for { |  | ||||||
| 		i++ |  | ||||||
| 		if len(datas) > l.batchNumber || (len(datas) > 0 && i > 3) { |  | ||||||
| 			_ = l.indexer.Index(datas) |  | ||||||
| 			datas = make([]*IndexerData, 0, l.batchNumber) |  | ||||||
| 			i = 0 |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		bs, err := l.queue.RPop() |  | ||||||
| 		if err != nil { |  | ||||||
| 			if err != levelqueue.ErrNotFound { |  | ||||||
| 				log.Error("RPop: %v", err) |  | ||||||
| 			} |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		if len(bs) == 0 { |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		var data IndexerData |  | ||||||
| 		err = json.Unmarshal(bs, &data) |  | ||||||
| 		if err != nil { |  | ||||||
| 			log.Error("Unmarshal: %v", err) |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		log.Trace("LevelQueue: task found: %#v", data) |  | ||||||
|  |  | ||||||
| 		if data.IsDelete { |  | ||||||
| 			if data.ID > 0 { |  | ||||||
| 				if err = l.indexer.Delete(data.ID); err != nil { |  | ||||||
| 					log.Error("indexer.Delete: %v", err) |  | ||||||
| 				} |  | ||||||
| 			} else if len(data.IDs) > 0 { |  | ||||||
| 				if err = l.indexer.Delete(data.IDs...); err != nil { |  | ||||||
| 					log.Error("indexer.Delete: %v", err) |  | ||||||
| 				} |  | ||||||
| 			} |  | ||||||
| 			time.Sleep(time.Millisecond * 10) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		datas = append(datas, &data) |  | ||||||
| 		time.Sleep(time.Millisecond * 10) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Push will push the indexer data to queue |  | ||||||
| func (l *LevelQueue) Push(data *IndexerData) error { |  | ||||||
| 	bs, err := json.Marshal(data) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return err |  | ||||||
| 	} |  | ||||||
| 	return l.queue.LPush(bs) |  | ||||||
| } |  | ||||||
| @@ -1,146 +0,0 @@ | |||||||
| // 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 issues |  | ||||||
|  |  | ||||||
| import ( |  | ||||||
| 	"encoding/json" |  | ||||||
| 	"errors" |  | ||||||
| 	"strconv" |  | ||||||
| 	"strings" |  | ||||||
| 	"time" |  | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/modules/log" |  | ||||||
|  |  | ||||||
| 	"github.com/go-redis/redis" |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| var ( |  | ||||||
| 	_ Queue = &RedisQueue{} |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| type redisClient interface { |  | ||||||
| 	RPush(key string, args ...interface{}) *redis.IntCmd |  | ||||||
| 	LPop(key string) *redis.StringCmd |  | ||||||
| 	Ping() *redis.StatusCmd |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // RedisQueue redis queue |  | ||||||
| type RedisQueue struct { |  | ||||||
| 	client      redisClient |  | ||||||
| 	queueName   string |  | ||||||
| 	indexer     Indexer |  | ||||||
| 	batchNumber int |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func parseConnStr(connStr string) (addrs, password string, dbIdx int, err error) { |  | ||||||
| 	fields := strings.Fields(connStr) |  | ||||||
| 	for _, f := range fields { |  | ||||||
| 		items := strings.SplitN(f, "=", 2) |  | ||||||
| 		if len(items) < 2 { |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
| 		switch strings.ToLower(items[0]) { |  | ||||||
| 		case "addrs": |  | ||||||
| 			addrs = items[1] |  | ||||||
| 		case "password": |  | ||||||
| 			password = items[1] |  | ||||||
| 		case "db": |  | ||||||
| 			dbIdx, err = strconv.Atoi(items[1]) |  | ||||||
| 			if err != nil { |  | ||||||
| 				return |  | ||||||
| 			} |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| 	return |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // NewRedisQueue creates single redis or cluster redis queue |  | ||||||
| func NewRedisQueue(addrs string, password string, dbIdx int, indexer Indexer, batchNumber int) (*RedisQueue, error) { |  | ||||||
| 	dbs := strings.Split(addrs, ",") |  | ||||||
| 	var queue = RedisQueue{ |  | ||||||
| 		queueName:   "issue_indexer_queue", |  | ||||||
| 		indexer:     indexer, |  | ||||||
| 		batchNumber: batchNumber, |  | ||||||
| 	} |  | ||||||
| 	if len(dbs) == 0 { |  | ||||||
| 		return nil, errors.New("no redis host found") |  | ||||||
| 	} else if len(dbs) == 1 { |  | ||||||
| 		queue.client = redis.NewClient(&redis.Options{ |  | ||||||
| 			Addr:     strings.TrimSpace(dbs[0]), // use default Addr |  | ||||||
| 			Password: password,                  // no password set |  | ||||||
| 			DB:       dbIdx,                     // use default DB |  | ||||||
| 		}) |  | ||||||
| 	} else { |  | ||||||
| 		queue.client = redis.NewClusterClient(&redis.ClusterOptions{ |  | ||||||
| 			Addrs: dbs, |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| 	if err := queue.client.Ping().Err(); err != nil { |  | ||||||
| 		return nil, err |  | ||||||
| 	} |  | ||||||
| 	return &queue, nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Run runs the redis queue |  | ||||||
| func (r *RedisQueue) Run() error { |  | ||||||
| 	var i int |  | ||||||
| 	var datas = make([]*IndexerData, 0, r.batchNumber) |  | ||||||
| 	for { |  | ||||||
| 		bs, err := r.client.LPop(r.queueName).Bytes() |  | ||||||
| 		if err != nil && err != redis.Nil { |  | ||||||
| 			log.Error("LPop faile: %v", err) |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		i++ |  | ||||||
| 		if len(datas) > r.batchNumber || (len(datas) > 0 && i > 3) { |  | ||||||
| 			_ = r.indexer.Index(datas) |  | ||||||
| 			datas = make([]*IndexerData, 0, r.batchNumber) |  | ||||||
| 			i = 0 |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		if len(bs) == 0 { |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		var data IndexerData |  | ||||||
| 		err = json.Unmarshal(bs, &data) |  | ||||||
| 		if err != nil { |  | ||||||
| 			log.Error("Unmarshal: %v", err) |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		log.Trace("RedisQueue: task found: %#v", data) |  | ||||||
|  |  | ||||||
| 		if data.IsDelete { |  | ||||||
| 			if data.ID > 0 { |  | ||||||
| 				if err = r.indexer.Delete(data.ID); err != nil { |  | ||||||
| 					log.Error("indexer.Delete: %v", err) |  | ||||||
| 				} |  | ||||||
| 			} else if len(data.IDs) > 0 { |  | ||||||
| 				if err = r.indexer.Delete(data.IDs...); err != nil { |  | ||||||
| 					log.Error("indexer.Delete: %v", err) |  | ||||||
| 				} |  | ||||||
| 			} |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		datas = append(datas, &data) |  | ||||||
| 		time.Sleep(time.Millisecond * 100) |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Push implements Queue |  | ||||||
| func (r *RedisQueue) Push(data *IndexerData) error { |  | ||||||
| 	bs, err := json.Marshal(data) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return err |  | ||||||
| 	} |  | ||||||
| 	return r.client.RPush(r.queueName, bs).Err() |  | ||||||
| } |  | ||||||
							
								
								
									
										270
									
								
								modules/queue/manager.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										270
									
								
								modules/queue/manager.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,270 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"encoding/json" | ||||||
|  | 	"fmt" | ||||||
|  | 	"reflect" | ||||||
|  | 	"sort" | ||||||
|  | 	"sync" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | var manager *Manager | ||||||
|  |  | ||||||
|  | // Manager is a queue manager | ||||||
|  | type Manager struct { | ||||||
|  | 	mutex sync.Mutex | ||||||
|  |  | ||||||
|  | 	counter int64 | ||||||
|  | 	Queues  map[int64]*ManagedQueue | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // ManagedQueue represents a working queue inheriting from Gitea. | ||||||
|  | type ManagedQueue struct { | ||||||
|  | 	mutex         sync.Mutex | ||||||
|  | 	QID           int64 | ||||||
|  | 	Queue         Queue | ||||||
|  | 	Type          Type | ||||||
|  | 	Name          string | ||||||
|  | 	Configuration interface{} | ||||||
|  | 	ExemplarType  string | ||||||
|  | 	Pool          ManagedPool | ||||||
|  | 	counter       int64 | ||||||
|  | 	PoolWorkers   map[int64]*PoolWorkers | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // ManagedPool is a simple interface to get certain details from a worker pool | ||||||
|  | type ManagedPool interface { | ||||||
|  | 	AddWorkers(number int, timeout time.Duration) context.CancelFunc | ||||||
|  | 	NumberOfWorkers() int | ||||||
|  | 	MaxNumberOfWorkers() int | ||||||
|  | 	SetMaxNumberOfWorkers(int) | ||||||
|  | 	BoostTimeout() time.Duration | ||||||
|  | 	BlockTimeout() time.Duration | ||||||
|  | 	BoostWorkers() int | ||||||
|  | 	SetSettings(maxNumberOfWorkers, boostWorkers int, timeout time.Duration) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // ManagedQueueList implements the sort.Interface | ||||||
|  | type ManagedQueueList []*ManagedQueue | ||||||
|  |  | ||||||
|  | // PoolWorkers represents a working queue inheriting from Gitea. | ||||||
|  | type PoolWorkers struct { | ||||||
|  | 	PID        int64 | ||||||
|  | 	Workers    int | ||||||
|  | 	Start      time.Time | ||||||
|  | 	Timeout    time.Time | ||||||
|  | 	HasTimeout bool | ||||||
|  | 	Cancel     context.CancelFunc | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // PoolWorkersList implements the sort.Interface | ||||||
|  | type PoolWorkersList []*PoolWorkers | ||||||
|  |  | ||||||
|  | func init() { | ||||||
|  | 	_ = GetManager() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // GetManager returns a Manager and initializes one as singleton if there's none yet | ||||||
|  | func GetManager() *Manager { | ||||||
|  | 	if manager == nil { | ||||||
|  | 		manager = &Manager{ | ||||||
|  | 			Queues: make(map[int64]*ManagedQueue), | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return manager | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Add adds a queue to this manager | ||||||
|  | func (m *Manager) Add(queue Queue, | ||||||
|  | 	t Type, | ||||||
|  | 	configuration, | ||||||
|  | 	exemplar interface{}, | ||||||
|  | 	pool ManagedPool) int64 { | ||||||
|  |  | ||||||
|  | 	cfg, _ := json.Marshal(configuration) | ||||||
|  | 	mq := &ManagedQueue{ | ||||||
|  | 		Queue:         queue, | ||||||
|  | 		Type:          t, | ||||||
|  | 		Configuration: string(cfg), | ||||||
|  | 		ExemplarType:  reflect.TypeOf(exemplar).String(), | ||||||
|  | 		PoolWorkers:   make(map[int64]*PoolWorkers), | ||||||
|  | 		Pool:          pool, | ||||||
|  | 	} | ||||||
|  | 	m.mutex.Lock() | ||||||
|  | 	m.counter++ | ||||||
|  | 	mq.QID = m.counter | ||||||
|  | 	mq.Name = fmt.Sprintf("queue-%d", mq.QID) | ||||||
|  | 	if named, ok := queue.(Named); ok { | ||||||
|  | 		mq.Name = named.Name() | ||||||
|  | 	} | ||||||
|  | 	m.Queues[mq.QID] = mq | ||||||
|  | 	m.mutex.Unlock() | ||||||
|  | 	log.Trace("Queue Manager registered: %s (QID: %d)", mq.Name, mq.QID) | ||||||
|  | 	return mq.QID | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Remove a queue from the Manager | ||||||
|  | func (m *Manager) Remove(qid int64) { | ||||||
|  | 	m.mutex.Lock() | ||||||
|  | 	delete(m.Queues, qid) | ||||||
|  | 	m.mutex.Unlock() | ||||||
|  | 	log.Trace("Queue Manager removed: QID: %d", qid) | ||||||
|  |  | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // GetManagedQueue by qid | ||||||
|  | func (m *Manager) GetManagedQueue(qid int64) *ManagedQueue { | ||||||
|  | 	m.mutex.Lock() | ||||||
|  | 	defer m.mutex.Unlock() | ||||||
|  | 	return m.Queues[qid] | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // ManagedQueues returns the managed queues | ||||||
|  | func (m *Manager) ManagedQueues() []*ManagedQueue { | ||||||
|  | 	m.mutex.Lock() | ||||||
|  | 	mqs := make([]*ManagedQueue, 0, len(m.Queues)) | ||||||
|  | 	for _, mq := range m.Queues { | ||||||
|  | 		mqs = append(mqs, mq) | ||||||
|  | 	} | ||||||
|  | 	m.mutex.Unlock() | ||||||
|  | 	sort.Sort(ManagedQueueList(mqs)) | ||||||
|  | 	return mqs | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Workers returns the poolworkers | ||||||
|  | func (q *ManagedQueue) Workers() []*PoolWorkers { | ||||||
|  | 	q.mutex.Lock() | ||||||
|  | 	workers := make([]*PoolWorkers, 0, len(q.PoolWorkers)) | ||||||
|  | 	for _, worker := range q.PoolWorkers { | ||||||
|  | 		workers = append(workers, worker) | ||||||
|  | 	} | ||||||
|  | 	q.mutex.Unlock() | ||||||
|  | 	sort.Sort(PoolWorkersList(workers)) | ||||||
|  | 	return workers | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // RegisterWorkers registers workers to this queue | ||||||
|  | func (q *ManagedQueue) RegisterWorkers(number int, start time.Time, hasTimeout bool, timeout time.Time, cancel context.CancelFunc) int64 { | ||||||
|  | 	q.mutex.Lock() | ||||||
|  | 	defer q.mutex.Unlock() | ||||||
|  | 	q.counter++ | ||||||
|  | 	q.PoolWorkers[q.counter] = &PoolWorkers{ | ||||||
|  | 		PID:        q.counter, | ||||||
|  | 		Workers:    number, | ||||||
|  | 		Start:      start, | ||||||
|  | 		Timeout:    timeout, | ||||||
|  | 		HasTimeout: hasTimeout, | ||||||
|  | 		Cancel:     cancel, | ||||||
|  | 	} | ||||||
|  | 	return q.counter | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // CancelWorkers cancels pooled workers with pid | ||||||
|  | func (q *ManagedQueue) CancelWorkers(pid int64) { | ||||||
|  | 	q.mutex.Lock() | ||||||
|  | 	pw, ok := q.PoolWorkers[pid] | ||||||
|  | 	q.mutex.Unlock() | ||||||
|  | 	if !ok { | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	pw.Cancel() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // RemoveWorkers deletes pooled workers with pid | ||||||
|  | func (q *ManagedQueue) RemoveWorkers(pid int64) { | ||||||
|  | 	q.mutex.Lock() | ||||||
|  | 	pw, ok := q.PoolWorkers[pid] | ||||||
|  | 	delete(q.PoolWorkers, pid) | ||||||
|  | 	q.mutex.Unlock() | ||||||
|  | 	if ok && pw.Cancel != nil { | ||||||
|  | 		pw.Cancel() | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // AddWorkers adds workers to the queue if it has registered an add worker function | ||||||
|  | func (q *ManagedQueue) AddWorkers(number int, timeout time.Duration) context.CancelFunc { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		// the cancel will be added to the pool workers description above | ||||||
|  | 		return q.Pool.AddWorkers(number, timeout) | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NumberOfWorkers returns the number of workers in the queue | ||||||
|  | func (q *ManagedQueue) NumberOfWorkers() int { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		return q.Pool.NumberOfWorkers() | ||||||
|  | 	} | ||||||
|  | 	return -1 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // MaxNumberOfWorkers returns the maximum number of workers for the pool | ||||||
|  | func (q *ManagedQueue) MaxNumberOfWorkers() int { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		return q.Pool.MaxNumberOfWorkers() | ||||||
|  | 	} | ||||||
|  | 	return 0 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // BoostWorkers returns the number of workers for a boost | ||||||
|  | func (q *ManagedQueue) BoostWorkers() int { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		return q.Pool.BoostWorkers() | ||||||
|  | 	} | ||||||
|  | 	return -1 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // BoostTimeout returns the timeout of the next boost | ||||||
|  | func (q *ManagedQueue) BoostTimeout() time.Duration { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		return q.Pool.BoostTimeout() | ||||||
|  | 	} | ||||||
|  | 	return 0 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // BlockTimeout returns the timeout til the next boost | ||||||
|  | func (q *ManagedQueue) BlockTimeout() time.Duration { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		return q.Pool.BlockTimeout() | ||||||
|  | 	} | ||||||
|  | 	return 0 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // SetSettings sets the setable boost values | ||||||
|  | func (q *ManagedQueue) SetSettings(maxNumberOfWorkers, boostWorkers int, timeout time.Duration) { | ||||||
|  | 	if q.Pool != nil { | ||||||
|  | 		q.Pool.SetSettings(maxNumberOfWorkers, boostWorkers, timeout) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l ManagedQueueList) Len() int { | ||||||
|  | 	return len(l) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l ManagedQueueList) Less(i, j int) bool { | ||||||
|  | 	return l[i].Name < l[j].Name | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l ManagedQueueList) Swap(i, j int) { | ||||||
|  | 	l[i], l[j] = l[j], l[i] | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l PoolWorkersList) Len() int { | ||||||
|  | 	return len(l) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l PoolWorkersList) Less(i, j int) bool { | ||||||
|  | 	return l[i].Start.Before(l[j].Start) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l PoolWorkersList) Swap(i, j int) { | ||||||
|  | 	l[i], l[j] = l[j], l[i] | ||||||
|  | } | ||||||
							
								
								
									
										133
									
								
								modules/queue/queue.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										133
									
								
								modules/queue/queue.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,133 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"encoding/json" | ||||||
|  | 	"fmt" | ||||||
|  | 	"reflect" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // ErrInvalidConfiguration is called when there is invalid configuration for a queue | ||||||
|  | type ErrInvalidConfiguration struct { | ||||||
|  | 	cfg interface{} | ||||||
|  | 	err error | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (err ErrInvalidConfiguration) Error() string { | ||||||
|  | 	if err.err != nil { | ||||||
|  | 		return fmt.Sprintf("Invalid Configuration Argument: %v: Error: %v", err.cfg, err.err) | ||||||
|  | 	} | ||||||
|  | 	return fmt.Sprintf("Invalid Configuration Argument: %v", err.cfg) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // IsErrInvalidConfiguration checks if an error is an ErrInvalidConfiguration | ||||||
|  | func IsErrInvalidConfiguration(err error) bool { | ||||||
|  | 	_, ok := err.(ErrInvalidConfiguration) | ||||||
|  | 	return ok | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Type is a type of Queue | ||||||
|  | type Type string | ||||||
|  |  | ||||||
|  | // Data defines an type of queuable data | ||||||
|  | type Data interface{} | ||||||
|  |  | ||||||
|  | // HandlerFunc is a function that takes a variable amount of data and processes it | ||||||
|  | type HandlerFunc func(...Data) | ||||||
|  |  | ||||||
|  | // NewQueueFunc is a function that creates a queue | ||||||
|  | type NewQueueFunc func(handler HandlerFunc, config interface{}, exemplar interface{}) (Queue, error) | ||||||
|  |  | ||||||
|  | // Shutdownable represents a queue that can be shutdown | ||||||
|  | type Shutdownable interface { | ||||||
|  | 	Shutdown() | ||||||
|  | 	Terminate() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Named represents a queue with a name | ||||||
|  | type Named interface { | ||||||
|  | 	Name() string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Queue defines an interface to save an issue indexer queue | ||||||
|  | type Queue interface { | ||||||
|  | 	Run(atShutdown, atTerminate func(context.Context, func())) | ||||||
|  | 	Push(Data) error | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // DummyQueueType is the type for the dummy queue | ||||||
|  | const DummyQueueType Type = "dummy" | ||||||
|  |  | ||||||
|  | // NewDummyQueue creates a new DummyQueue | ||||||
|  | func NewDummyQueue(handler HandlerFunc, opts, exemplar interface{}) (Queue, error) { | ||||||
|  | 	return &DummyQueue{}, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // DummyQueue represents an empty queue | ||||||
|  | type DummyQueue struct { | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run starts to run the queue | ||||||
|  | func (b *DummyQueue) Run(_, _ func(context.Context, func())) {} | ||||||
|  |  | ||||||
|  | // Push pushes data to the queue | ||||||
|  | func (b *DummyQueue) Push(Data) error { | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func toConfig(exemplar, cfg interface{}) (interface{}, error) { | ||||||
|  | 	if reflect.TypeOf(cfg).AssignableTo(reflect.TypeOf(exemplar)) { | ||||||
|  | 		return cfg, nil | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	configBytes, ok := cfg.([]byte) | ||||||
|  | 	if !ok { | ||||||
|  | 		configStr, ok := cfg.(string) | ||||||
|  | 		if !ok { | ||||||
|  | 			return nil, ErrInvalidConfiguration{cfg: cfg} | ||||||
|  | 		} | ||||||
|  | 		configBytes = []byte(configStr) | ||||||
|  | 	} | ||||||
|  | 	newVal := reflect.New(reflect.TypeOf(exemplar)) | ||||||
|  | 	if err := json.Unmarshal(configBytes, newVal.Interface()); err != nil { | ||||||
|  | 		return nil, ErrInvalidConfiguration{cfg: cfg, err: err} | ||||||
|  | 	} | ||||||
|  | 	return newVal.Elem().Interface(), nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | var queuesMap = map[Type]NewQueueFunc{DummyQueueType: NewDummyQueue} | ||||||
|  |  | ||||||
|  | // RegisteredTypes provides the list of requested types of queues | ||||||
|  | func RegisteredTypes() []Type { | ||||||
|  | 	types := make([]Type, len(queuesMap)) | ||||||
|  | 	i := 0 | ||||||
|  | 	for key := range queuesMap { | ||||||
|  | 		types[i] = key | ||||||
|  | 		i++ | ||||||
|  | 	} | ||||||
|  | 	return types | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // RegisteredTypesAsString provides the list of requested types of queues | ||||||
|  | func RegisteredTypesAsString() []string { | ||||||
|  | 	types := make([]string, len(queuesMap)) | ||||||
|  | 	i := 0 | ||||||
|  | 	for key := range queuesMap { | ||||||
|  | 		types[i] = string(key) | ||||||
|  | 		i++ | ||||||
|  | 	} | ||||||
|  | 	return types | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewQueue takes a queue Type and HandlerFunc some options and possibly an exemplar and returns a Queue or an error | ||||||
|  | func NewQueue(queueType Type, handlerFunc HandlerFunc, opts, exemplar interface{}) (Queue, error) { | ||||||
|  | 	newFn, ok := queuesMap[queueType] | ||||||
|  | 	if !ok { | ||||||
|  | 		return nil, fmt.Errorf("Unsupported queue type: %v", queueType) | ||||||
|  | 	} | ||||||
|  | 	return newFn(handlerFunc, opts, exemplar) | ||||||
|  | } | ||||||
							
								
								
									
										106
									
								
								modules/queue/queue_channel.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										106
									
								
								modules/queue/queue_channel.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,106 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 	"reflect" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // ChannelQueueType is the type for channel queue | ||||||
|  | const ChannelQueueType Type = "channel" | ||||||
|  |  | ||||||
|  | // ChannelQueueConfiguration is the configuration for a ChannelQueue | ||||||
|  | type ChannelQueueConfiguration struct { | ||||||
|  | 	QueueLength  int | ||||||
|  | 	BatchLength  int | ||||||
|  | 	Workers      int | ||||||
|  | 	MaxWorkers   int | ||||||
|  | 	BlockTimeout time.Duration | ||||||
|  | 	BoostTimeout time.Duration | ||||||
|  | 	BoostWorkers int | ||||||
|  | 	Name         string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // ChannelQueue implements | ||||||
|  | type ChannelQueue struct { | ||||||
|  | 	pool     *WorkerPool | ||||||
|  | 	exemplar interface{} | ||||||
|  | 	workers  int | ||||||
|  | 	name     string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewChannelQueue create a memory channel queue | ||||||
|  | func NewChannelQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | ||||||
|  | 	configInterface, err := toConfig(ChannelQueueConfiguration{}, cfg) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	config := configInterface.(ChannelQueueConfiguration) | ||||||
|  | 	if config.BatchLength == 0 { | ||||||
|  | 		config.BatchLength = 1 | ||||||
|  | 	} | ||||||
|  | 	dataChan := make(chan Data, config.QueueLength) | ||||||
|  |  | ||||||
|  | 	ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  | 	queue := &ChannelQueue{ | ||||||
|  | 		pool: &WorkerPool{ | ||||||
|  | 			baseCtx:            ctx, | ||||||
|  | 			cancel:             cancel, | ||||||
|  | 			batchLength:        config.BatchLength, | ||||||
|  | 			handle:             handle, | ||||||
|  | 			dataChan:           dataChan, | ||||||
|  | 			blockTimeout:       config.BlockTimeout, | ||||||
|  | 			boostTimeout:       config.BoostTimeout, | ||||||
|  | 			boostWorkers:       config.BoostWorkers, | ||||||
|  | 			maxNumberOfWorkers: config.MaxWorkers, | ||||||
|  | 		}, | ||||||
|  | 		exemplar: exemplar, | ||||||
|  | 		workers:  config.Workers, | ||||||
|  | 		name:     config.Name, | ||||||
|  | 	} | ||||||
|  | 	queue.pool.qid = GetManager().Add(queue, ChannelQueueType, config, exemplar, queue.pool) | ||||||
|  | 	return queue, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run starts to run the queue | ||||||
|  | func (c *ChannelQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | ||||||
|  | 	atShutdown(context.Background(), func() { | ||||||
|  | 		log.Warn("ChannelQueue: %s is not shutdownable!", c.name) | ||||||
|  | 	}) | ||||||
|  | 	atTerminate(context.Background(), func() { | ||||||
|  | 		log.Warn("ChannelQueue: %s is not terminatable!", c.name) | ||||||
|  | 	}) | ||||||
|  | 	go func() { | ||||||
|  | 		_ = c.pool.AddWorkers(c.workers, 0) | ||||||
|  | 	}() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Push will push data into the queue | ||||||
|  | func (c *ChannelQueue) Push(data Data) error { | ||||||
|  | 	if c.exemplar != nil { | ||||||
|  | 		// Assert data is of same type as r.exemplar | ||||||
|  | 		t := reflect.TypeOf(data) | ||||||
|  | 		exemplarType := reflect.TypeOf(c.exemplar) | ||||||
|  | 		if !t.AssignableTo(exemplarType) || data == nil { | ||||||
|  | 			return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in queue: %s", data, c.exemplar, c.name) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	c.pool.Push(data) | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Name returns the name of this queue | ||||||
|  | func (c *ChannelQueue) Name() string { | ||||||
|  | 	return c.name | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func init() { | ||||||
|  | 	queuesMap[ChannelQueueType] = NewChannelQueue | ||||||
|  | } | ||||||
							
								
								
									
										91
									
								
								modules/queue/queue_channel_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										91
									
								
								modules/queue/queue_channel_test.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,91 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"testing" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | func TestChannelQueue(t *testing.T) { | ||||||
|  | 	handleChan := make(chan *testData) | ||||||
|  | 	handle := func(data ...Data) { | ||||||
|  | 		for _, datum := range data { | ||||||
|  | 			testDatum := datum.(*testData) | ||||||
|  | 			handleChan <- testDatum | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	nilFn := func(_ context.Context, _ func()) {} | ||||||
|  |  | ||||||
|  | 	queue, err := NewChannelQueue(handle, | ||||||
|  | 		ChannelQueueConfiguration{ | ||||||
|  | 			QueueLength:  20, | ||||||
|  | 			Workers:      1, | ||||||
|  | 			MaxWorkers:   10, | ||||||
|  | 			BlockTimeout: 1 * time.Second, | ||||||
|  | 			BoostTimeout: 5 * time.Minute, | ||||||
|  | 			BoostWorkers: 5, | ||||||
|  | 		}, &testData{}) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	go queue.Run(nilFn, nilFn) | ||||||
|  |  | ||||||
|  | 	test1 := testData{"A", 1} | ||||||
|  | 	go queue.Push(&test1) | ||||||
|  | 	result1 := <-handleChan | ||||||
|  | 	assert.Equal(t, test1.TestString, result1.TestString) | ||||||
|  | 	assert.Equal(t, test1.TestInt, result1.TestInt) | ||||||
|  |  | ||||||
|  | 	err = queue.Push(test1) | ||||||
|  | 	assert.Error(t, err) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func TestChannelQueue_Batch(t *testing.T) { | ||||||
|  | 	handleChan := make(chan *testData) | ||||||
|  | 	handle := func(data ...Data) { | ||||||
|  | 		assert.True(t, len(data) == 2) | ||||||
|  | 		for _, datum := range data { | ||||||
|  | 			testDatum := datum.(*testData) | ||||||
|  | 			handleChan <- testDatum | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	nilFn := func(_ context.Context, _ func()) {} | ||||||
|  |  | ||||||
|  | 	queue, err := NewChannelQueue(handle, | ||||||
|  | 		ChannelQueueConfiguration{ | ||||||
|  | 			QueueLength:  20, | ||||||
|  | 			BatchLength:  2, | ||||||
|  | 			Workers:      1, | ||||||
|  | 			MaxWorkers:   10, | ||||||
|  | 			BlockTimeout: 1 * time.Second, | ||||||
|  | 			BoostTimeout: 5 * time.Minute, | ||||||
|  | 			BoostWorkers: 5, | ||||||
|  | 		}, &testData{}) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	go queue.Run(nilFn, nilFn) | ||||||
|  |  | ||||||
|  | 	test1 := testData{"A", 1} | ||||||
|  | 	test2 := testData{"B", 2} | ||||||
|  |  | ||||||
|  | 	queue.Push(&test1) | ||||||
|  | 	go queue.Push(&test2) | ||||||
|  |  | ||||||
|  | 	result1 := <-handleChan | ||||||
|  | 	assert.Equal(t, test1.TestString, result1.TestString) | ||||||
|  | 	assert.Equal(t, test1.TestInt, result1.TestInt) | ||||||
|  |  | ||||||
|  | 	result2 := <-handleChan | ||||||
|  | 	assert.Equal(t, test2.TestString, result2.TestString) | ||||||
|  | 	assert.Equal(t, test2.TestInt, result2.TestInt) | ||||||
|  |  | ||||||
|  | 	err = queue.Push(test1) | ||||||
|  | 	assert.Error(t, err) | ||||||
|  | } | ||||||
							
								
								
									
										213
									
								
								modules/queue/queue_disk.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										213
									
								
								modules/queue/queue_disk.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,213 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"encoding/json" | ||||||
|  | 	"fmt" | ||||||
|  | 	"reflect" | ||||||
|  | 	"sync" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  |  | ||||||
|  | 	"gitea.com/lunny/levelqueue" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // LevelQueueType is the type for level queue | ||||||
|  | const LevelQueueType Type = "level" | ||||||
|  |  | ||||||
|  | // LevelQueueConfiguration is the configuration for a LevelQueue | ||||||
|  | type LevelQueueConfiguration struct { | ||||||
|  | 	DataDir      string | ||||||
|  | 	QueueLength  int | ||||||
|  | 	BatchLength  int | ||||||
|  | 	Workers      int | ||||||
|  | 	MaxWorkers   int | ||||||
|  | 	BlockTimeout time.Duration | ||||||
|  | 	BoostTimeout time.Duration | ||||||
|  | 	BoostWorkers int | ||||||
|  | 	Name         string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // LevelQueue implements a disk library queue | ||||||
|  | type LevelQueue struct { | ||||||
|  | 	pool       *WorkerPool | ||||||
|  | 	queue      *levelqueue.Queue | ||||||
|  | 	closed     chan struct{} | ||||||
|  | 	terminated chan struct{} | ||||||
|  | 	lock       sync.Mutex | ||||||
|  | 	exemplar   interface{} | ||||||
|  | 	workers    int | ||||||
|  | 	name       string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewLevelQueue creates a ledis local queue | ||||||
|  | func NewLevelQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | ||||||
|  | 	configInterface, err := toConfig(LevelQueueConfiguration{}, cfg) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	config := configInterface.(LevelQueueConfiguration) | ||||||
|  |  | ||||||
|  | 	internal, err := levelqueue.Open(config.DataDir) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	dataChan := make(chan Data, config.QueueLength) | ||||||
|  | 	ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  |  | ||||||
|  | 	queue := &LevelQueue{ | ||||||
|  | 		pool: &WorkerPool{ | ||||||
|  | 			baseCtx:            ctx, | ||||||
|  | 			cancel:             cancel, | ||||||
|  | 			batchLength:        config.BatchLength, | ||||||
|  | 			handle:             handle, | ||||||
|  | 			dataChan:           dataChan, | ||||||
|  | 			blockTimeout:       config.BlockTimeout, | ||||||
|  | 			boostTimeout:       config.BoostTimeout, | ||||||
|  | 			boostWorkers:       config.BoostWorkers, | ||||||
|  | 			maxNumberOfWorkers: config.MaxWorkers, | ||||||
|  | 		}, | ||||||
|  | 		queue:      internal, | ||||||
|  | 		exemplar:   exemplar, | ||||||
|  | 		closed:     make(chan struct{}), | ||||||
|  | 		terminated: make(chan struct{}), | ||||||
|  | 		workers:    config.Workers, | ||||||
|  | 		name:       config.Name, | ||||||
|  | 	} | ||||||
|  | 	queue.pool.qid = GetManager().Add(queue, LevelQueueType, config, exemplar, queue.pool) | ||||||
|  | 	return queue, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run starts to run the queue | ||||||
|  | func (l *LevelQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | ||||||
|  | 	atShutdown(context.Background(), l.Shutdown) | ||||||
|  | 	atTerminate(context.Background(), l.Terminate) | ||||||
|  |  | ||||||
|  | 	go func() { | ||||||
|  | 		_ = l.pool.AddWorkers(l.workers, 0) | ||||||
|  | 	}() | ||||||
|  |  | ||||||
|  | 	go l.readToChan() | ||||||
|  |  | ||||||
|  | 	log.Trace("LevelQueue: %s Waiting til closed", l.name) | ||||||
|  | 	<-l.closed | ||||||
|  |  | ||||||
|  | 	log.Trace("LevelQueue: %s Waiting til done", l.name) | ||||||
|  | 	l.pool.Wait() | ||||||
|  |  | ||||||
|  | 	log.Trace("LevelQueue: %s Waiting til cleaned", l.name) | ||||||
|  | 	ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  | 	atTerminate(ctx, cancel) | ||||||
|  | 	l.pool.CleanUp(ctx) | ||||||
|  | 	cancel() | ||||||
|  | 	log.Trace("LevelQueue: %s Cleaned", l.name) | ||||||
|  |  | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (l *LevelQueue) readToChan() { | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case <-l.closed: | ||||||
|  | 			// tell the pool to shutdown. | ||||||
|  | 			l.pool.cancel() | ||||||
|  | 			return | ||||||
|  | 		default: | ||||||
|  | 			bs, err := l.queue.RPop() | ||||||
|  | 			if err != nil { | ||||||
|  | 				if err != levelqueue.ErrNotFound { | ||||||
|  | 					log.Error("LevelQueue: %s Error on RPop: %v", l.name, err) | ||||||
|  | 				} | ||||||
|  | 				time.Sleep(time.Millisecond * 100) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			if len(bs) == 0 { | ||||||
|  | 				time.Sleep(time.Millisecond * 100) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			var data Data | ||||||
|  | 			if l.exemplar != nil { | ||||||
|  | 				t := reflect.TypeOf(l.exemplar) | ||||||
|  | 				n := reflect.New(t) | ||||||
|  | 				ne := n.Elem() | ||||||
|  | 				err = json.Unmarshal(bs, ne.Addr().Interface()) | ||||||
|  | 				data = ne.Interface().(Data) | ||||||
|  | 			} else { | ||||||
|  | 				err = json.Unmarshal(bs, &data) | ||||||
|  | 			} | ||||||
|  | 			if err != nil { | ||||||
|  | 				log.Error("LevelQueue: %s Failed to unmarshal with error: %v", l.name, err) | ||||||
|  | 				time.Sleep(time.Millisecond * 100) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			log.Trace("LevelQueue %s: Task found: %#v", l.name, data) | ||||||
|  | 			l.pool.Push(data) | ||||||
|  |  | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Push will push the indexer data to queue | ||||||
|  | func (l *LevelQueue) Push(data Data) error { | ||||||
|  | 	if l.exemplar != nil { | ||||||
|  | 		// Assert data is of same type as r.exemplar | ||||||
|  | 		value := reflect.ValueOf(data) | ||||||
|  | 		t := value.Type() | ||||||
|  | 		exemplarType := reflect.ValueOf(l.exemplar).Type() | ||||||
|  | 		if !t.AssignableTo(exemplarType) || data == nil { | ||||||
|  | 			return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in %s", data, l.exemplar, l.name) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	bs, err := json.Marshal(data) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  | 	return l.queue.LPush(bs) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Shutdown this queue and stop processing | ||||||
|  | func (l *LevelQueue) Shutdown() { | ||||||
|  | 	l.lock.Lock() | ||||||
|  | 	defer l.lock.Unlock() | ||||||
|  | 	log.Trace("LevelQueue: %s Shutdown", l.name) | ||||||
|  | 	select { | ||||||
|  | 	case <-l.closed: | ||||||
|  | 	default: | ||||||
|  | 		close(l.closed) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Terminate this queue and close the queue | ||||||
|  | func (l *LevelQueue) Terminate() { | ||||||
|  | 	log.Trace("LevelQueue: %s Terminating", l.name) | ||||||
|  | 	l.Shutdown() | ||||||
|  | 	l.lock.Lock() | ||||||
|  | 	select { | ||||||
|  | 	case <-l.terminated: | ||||||
|  | 		l.lock.Unlock() | ||||||
|  | 	default: | ||||||
|  | 		close(l.terminated) | ||||||
|  | 		l.lock.Unlock() | ||||||
|  | 		if err := l.queue.Close(); err != nil && err.Error() != "leveldb: closed" { | ||||||
|  | 			log.Error("Error whilst closing internal queue in %s: %v", l.name, err) | ||||||
|  | 		} | ||||||
|  |  | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Name returns the name of this queue | ||||||
|  | func (l *LevelQueue) Name() string { | ||||||
|  | 	return l.name | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func init() { | ||||||
|  | 	queuesMap[LevelQueueType] = NewLevelQueue | ||||||
|  | } | ||||||
							
								
								
									
										193
									
								
								modules/queue/queue_disk_channel.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										193
									
								
								modules/queue/queue_disk_channel.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,193 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // PersistableChannelQueueType is the type for persistable queue | ||||||
|  | const PersistableChannelQueueType Type = "persistable-channel" | ||||||
|  |  | ||||||
|  | // PersistableChannelQueueConfiguration is the configuration for a PersistableChannelQueue | ||||||
|  | type PersistableChannelQueueConfiguration 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 | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // PersistableChannelQueue wraps a channel queue and level queue together | ||||||
|  | type PersistableChannelQueue struct { | ||||||
|  | 	*ChannelQueue | ||||||
|  | 	delayedStarter | ||||||
|  | 	closed chan struct{} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewPersistableChannelQueue 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 NewPersistableChannelQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | ||||||
|  | 	configInterface, err := toConfig(PersistableChannelQueueConfiguration{}, cfg) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	config := configInterface.(PersistableChannelQueueConfiguration) | ||||||
|  |  | ||||||
|  | 	channelQueue, err := NewChannelQueue(handle, ChannelQueueConfiguration{ | ||||||
|  | 		QueueLength:  config.QueueLength, | ||||||
|  | 		BatchLength:  config.BatchLength, | ||||||
|  | 		Workers:      config.Workers, | ||||||
|  | 		MaxWorkers:   config.MaxWorkers, | ||||||
|  | 		BlockTimeout: config.BlockTimeout, | ||||||
|  | 		BoostTimeout: config.BoostTimeout, | ||||||
|  | 		BoostWorkers: config.BoostWorkers, | ||||||
|  | 		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 := LevelQueueConfiguration{ | ||||||
|  | 		DataDir:      config.DataDir, | ||||||
|  | 		QueueLength:  config.QueueLength, | ||||||
|  | 		BatchLength:  config.BatchLength, | ||||||
|  | 		Workers:      1, | ||||||
|  | 		MaxWorkers:   6, | ||||||
|  | 		BlockTimeout: 1 * time.Second, | ||||||
|  | 		BoostTimeout: 5 * time.Minute, | ||||||
|  | 		BoostWorkers: 5, | ||||||
|  | 		Name:         config.Name + "-level", | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	levelQueue, err := NewLevelQueue(handle, levelCfg, exemplar) | ||||||
|  | 	if err == nil { | ||||||
|  | 		queue := &PersistableChannelQueue{ | ||||||
|  | 			ChannelQueue: channelQueue.(*ChannelQueue), | ||||||
|  | 			delayedStarter: delayedStarter{ | ||||||
|  | 				internal: levelQueue.(*LevelQueue), | ||||||
|  | 				name:     config.Name, | ||||||
|  | 			}, | ||||||
|  | 			closed: make(chan struct{}), | ||||||
|  | 		} | ||||||
|  | 		_ = GetManager().Add(queue, PersistableChannelQueueType, config, exemplar, nil) | ||||||
|  | 		return queue, nil | ||||||
|  | 	} | ||||||
|  | 	if IsErrInvalidConfiguration(err) { | ||||||
|  | 		// Retrying ain't gonna make this any better... | ||||||
|  | 		return nil, ErrInvalidConfiguration{cfg: cfg} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	queue := &PersistableChannelQueue{ | ||||||
|  | 		ChannelQueue: channelQueue.(*ChannelQueue), | ||||||
|  | 		delayedStarter: delayedStarter{ | ||||||
|  | 			cfg:         levelCfg, | ||||||
|  | 			underlying:  LevelQueueType, | ||||||
|  | 			timeout:     config.Timeout, | ||||||
|  | 			maxAttempts: config.MaxAttempts, | ||||||
|  | 			name:        config.Name, | ||||||
|  | 		}, | ||||||
|  | 		closed: make(chan struct{}), | ||||||
|  | 	} | ||||||
|  | 	_ = GetManager().Add(queue, PersistableChannelQueueType, config, exemplar, nil) | ||||||
|  | 	return queue, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Name returns the name of this queue | ||||||
|  | func (p *PersistableChannelQueue) Name() string { | ||||||
|  | 	return p.delayedStarter.name | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Push will push the indexer data to queue | ||||||
|  | func (p *PersistableChannelQueue) Push(data Data) error { | ||||||
|  | 	select { | ||||||
|  | 	case <-p.closed: | ||||||
|  | 		return p.internal.Push(data) | ||||||
|  | 	default: | ||||||
|  | 		return p.ChannelQueue.Push(data) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run starts to run the queue | ||||||
|  | func (p *PersistableChannelQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	if p.internal == nil { | ||||||
|  | 		err := p.setInternal(atShutdown, p.ChannelQueue.pool.handle, p.exemplar) | ||||||
|  | 		p.lock.Unlock() | ||||||
|  | 		if err != nil { | ||||||
|  | 			log.Fatal("Unable to create internal queue for %s Error: %v", p.Name(), err) | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 	} else { | ||||||
|  | 		p.lock.Unlock() | ||||||
|  | 	} | ||||||
|  | 	atShutdown(context.Background(), p.Shutdown) | ||||||
|  | 	atTerminate(context.Background(), p.Terminate) | ||||||
|  |  | ||||||
|  | 	// Just run the level queue - we shut it down later | ||||||
|  | 	go p.internal.Run(func(_ context.Context, _ func()) {}, func(_ context.Context, _ func()) {}) | ||||||
|  |  | ||||||
|  | 	go func() { | ||||||
|  | 		_ = p.ChannelQueue.pool.AddWorkers(p.workers, 0) | ||||||
|  | 	}() | ||||||
|  |  | ||||||
|  | 	log.Trace("PersistableChannelQueue: %s Waiting til closed", p.delayedStarter.name) | ||||||
|  | 	<-p.closed | ||||||
|  | 	log.Trace("PersistableChannelQueue: %s Cancelling pools", p.delayedStarter.name) | ||||||
|  | 	p.ChannelQueue.pool.cancel() | ||||||
|  | 	p.internal.(*LevelQueue).pool.cancel() | ||||||
|  | 	log.Trace("PersistableChannelQueue: %s Waiting til done", p.delayedStarter.name) | ||||||
|  | 	p.ChannelQueue.pool.Wait() | ||||||
|  | 	p.internal.(*LevelQueue).pool.Wait() | ||||||
|  | 	// Redirect all remaining data in the chan to the internal channel | ||||||
|  | 	go func() { | ||||||
|  | 		log.Trace("PersistableChannelQueue: %s Redirecting remaining data", p.delayedStarter.name) | ||||||
|  | 		for data := range p.ChannelQueue.pool.dataChan { | ||||||
|  | 			_ = p.internal.Push(data) | ||||||
|  | 		} | ||||||
|  | 		log.Trace("PersistableChannelQueue: %s Done Redirecting remaining data", p.delayedStarter.name) | ||||||
|  | 	}() | ||||||
|  | 	log.Trace("PersistableChannelQueue: %s Done main loop", p.delayedStarter.name) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Shutdown processing this queue | ||||||
|  | func (p *PersistableChannelQueue) Shutdown() { | ||||||
|  | 	log.Trace("PersistableChannelQueue: %s Shutdown", p.delayedStarter.name) | ||||||
|  | 	select { | ||||||
|  | 	case <-p.closed: | ||||||
|  | 	default: | ||||||
|  | 		p.lock.Lock() | ||||||
|  | 		defer p.lock.Unlock() | ||||||
|  | 		if p.internal != nil { | ||||||
|  | 			p.internal.(*LevelQueue).Shutdown() | ||||||
|  | 		} | ||||||
|  | 		close(p.closed) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Terminate this queue and close the queue | ||||||
|  | func (p *PersistableChannelQueue) Terminate() { | ||||||
|  | 	log.Trace("PersistableChannelQueue: %s Terminating", p.delayedStarter.name) | ||||||
|  | 	p.Shutdown() | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	if p.internal != nil { | ||||||
|  | 		p.internal.(*LevelQueue).Terminate() | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func init() { | ||||||
|  | 	queuesMap[PersistableChannelQueueType] = NewPersistableChannelQueue | ||||||
|  | } | ||||||
							
								
								
									
										117
									
								
								modules/queue/queue_disk_channel_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										117
									
								
								modules/queue/queue_disk_channel_test.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,117 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"io/ioutil" | ||||||
|  | 	"os" | ||||||
|  | 	"testing" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | func TestPersistableChannelQueue(t *testing.T) { | ||||||
|  | 	handleChan := make(chan *testData) | ||||||
|  | 	handle := func(data ...Data) { | ||||||
|  | 		assert.True(t, len(data) == 2) | ||||||
|  | 		for _, datum := range data { | ||||||
|  | 			testDatum := datum.(*testData) | ||||||
|  | 			handleChan <- testDatum | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	queueShutdown := []func(){} | ||||||
|  | 	queueTerminate := []func(){} | ||||||
|  |  | ||||||
|  | 	tmpDir, err := ioutil.TempDir("", "persistable-channel-queue-test-data") | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	defer os.RemoveAll(tmpDir) | ||||||
|  |  | ||||||
|  | 	queue, err := NewPersistableChannelQueue(handle, PersistableChannelQueueConfiguration{ | ||||||
|  | 		DataDir:     tmpDir, | ||||||
|  | 		BatchLength: 2, | ||||||
|  | 		QueueLength: 20, | ||||||
|  | 		Workers:     1, | ||||||
|  | 		MaxWorkers:  10, | ||||||
|  | 	}, &testData{}) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	go queue.Run(func(_ context.Context, shutdown func()) { | ||||||
|  | 		queueShutdown = append(queueShutdown, shutdown) | ||||||
|  | 	}, func(_ context.Context, terminate func()) { | ||||||
|  | 		queueTerminate = append(queueTerminate, terminate) | ||||||
|  | 	}) | ||||||
|  |  | ||||||
|  | 	test1 := testData{"A", 1} | ||||||
|  | 	test2 := testData{"B", 2} | ||||||
|  |  | ||||||
|  | 	err = queue.Push(&test1) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	go func() { | ||||||
|  | 		err = queue.Push(&test2) | ||||||
|  | 		assert.NoError(t, err) | ||||||
|  | 	}() | ||||||
|  |  | ||||||
|  | 	result1 := <-handleChan | ||||||
|  | 	assert.Equal(t, test1.TestString, result1.TestString) | ||||||
|  | 	assert.Equal(t, test1.TestInt, result1.TestInt) | ||||||
|  |  | ||||||
|  | 	result2 := <-handleChan | ||||||
|  | 	assert.Equal(t, test2.TestString, result2.TestString) | ||||||
|  | 	assert.Equal(t, test2.TestInt, result2.TestInt) | ||||||
|  |  | ||||||
|  | 	err = queue.Push(test1) | ||||||
|  | 	assert.Error(t, err) | ||||||
|  |  | ||||||
|  | 	for _, callback := range queueShutdown { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  | 	time.Sleep(200 * time.Millisecond) | ||||||
|  | 	err = queue.Push(&test1) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	err = queue.Push(&test2) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	select { | ||||||
|  | 	case <-handleChan: | ||||||
|  | 		assert.Fail(t, "Handler processing should have stopped") | ||||||
|  | 	default: | ||||||
|  | 	} | ||||||
|  | 	for _, callback := range queueTerminate { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// Reopen queue | ||||||
|  | 	queue, err = NewPersistableChannelQueue(handle, PersistableChannelQueueConfiguration{ | ||||||
|  | 		DataDir:     tmpDir, | ||||||
|  | 		BatchLength: 2, | ||||||
|  | 		QueueLength: 20, | ||||||
|  | 		Workers:     1, | ||||||
|  | 		MaxWorkers:  10, | ||||||
|  | 	}, &testData{}) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	go queue.Run(func(_ context.Context, shutdown func()) { | ||||||
|  | 		queueShutdown = append(queueShutdown, shutdown) | ||||||
|  | 	}, func(_ context.Context, terminate func()) { | ||||||
|  | 		queueTerminate = append(queueTerminate, terminate) | ||||||
|  | 	}) | ||||||
|  |  | ||||||
|  | 	result3 := <-handleChan | ||||||
|  | 	assert.Equal(t, test1.TestString, result3.TestString) | ||||||
|  | 	assert.Equal(t, test1.TestInt, result3.TestInt) | ||||||
|  |  | ||||||
|  | 	result4 := <-handleChan | ||||||
|  | 	assert.Equal(t, test2.TestString, result4.TestString) | ||||||
|  | 	assert.Equal(t, test2.TestInt, result4.TestInt) | ||||||
|  | 	for _, callback := range queueShutdown { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  | 	for _, callback := range queueTerminate { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | } | ||||||
							
								
								
									
										126
									
								
								modules/queue/queue_disk_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										126
									
								
								modules/queue/queue_disk_test.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,126 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"io/ioutil" | ||||||
|  | 	"os" | ||||||
|  | 	"testing" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | func TestLevelQueue(t *testing.T) { | ||||||
|  | 	handleChan := make(chan *testData) | ||||||
|  | 	handle := func(data ...Data) { | ||||||
|  | 		assert.True(t, len(data) == 2) | ||||||
|  | 		for _, datum := range data { | ||||||
|  | 			testDatum := datum.(*testData) | ||||||
|  | 			handleChan <- testDatum | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	queueShutdown := []func(){} | ||||||
|  | 	queueTerminate := []func(){} | ||||||
|  |  | ||||||
|  | 	tmpDir, err := ioutil.TempDir("", "level-queue-test-data") | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	defer os.RemoveAll(tmpDir) | ||||||
|  |  | ||||||
|  | 	queue, err := NewLevelQueue(handle, LevelQueueConfiguration{ | ||||||
|  | 		DataDir:      tmpDir, | ||||||
|  | 		BatchLength:  2, | ||||||
|  | 		Workers:      1, | ||||||
|  | 		MaxWorkers:   10, | ||||||
|  | 		QueueLength:  20, | ||||||
|  | 		BlockTimeout: 1 * time.Second, | ||||||
|  | 		BoostTimeout: 5 * time.Minute, | ||||||
|  | 		BoostWorkers: 5, | ||||||
|  | 	}, &testData{}) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	go queue.Run(func(_ context.Context, shutdown func()) { | ||||||
|  | 		queueShutdown = append(queueShutdown, shutdown) | ||||||
|  | 	}, func(_ context.Context, terminate func()) { | ||||||
|  | 		queueTerminate = append(queueTerminate, terminate) | ||||||
|  | 	}) | ||||||
|  |  | ||||||
|  | 	test1 := testData{"A", 1} | ||||||
|  | 	test2 := testData{"B", 2} | ||||||
|  |  | ||||||
|  | 	err = queue.Push(&test1) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	go func() { | ||||||
|  | 		err = queue.Push(&test2) | ||||||
|  | 		assert.NoError(t, err) | ||||||
|  | 	}() | ||||||
|  |  | ||||||
|  | 	result1 := <-handleChan | ||||||
|  | 	assert.Equal(t, test1.TestString, result1.TestString) | ||||||
|  | 	assert.Equal(t, test1.TestInt, result1.TestInt) | ||||||
|  |  | ||||||
|  | 	result2 := <-handleChan | ||||||
|  | 	assert.Equal(t, test2.TestString, result2.TestString) | ||||||
|  | 	assert.Equal(t, test2.TestInt, result2.TestInt) | ||||||
|  |  | ||||||
|  | 	err = queue.Push(test1) | ||||||
|  | 	assert.Error(t, err) | ||||||
|  |  | ||||||
|  | 	for _, callback := range queueShutdown { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  | 	time.Sleep(200 * time.Millisecond) | ||||||
|  | 	err = queue.Push(&test1) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	err = queue.Push(&test2) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	select { | ||||||
|  | 	case <-handleChan: | ||||||
|  | 		assert.Fail(t, "Handler processing should have stopped") | ||||||
|  | 	default: | ||||||
|  | 	} | ||||||
|  | 	for _, callback := range queueTerminate { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	// Reopen queue | ||||||
|  | 	queue, err = NewWrappedQueue(handle, | ||||||
|  | 		WrappedQueueConfiguration{ | ||||||
|  | 			Underlying: LevelQueueType, | ||||||
|  | 			Config: LevelQueueConfiguration{ | ||||||
|  | 				DataDir:      tmpDir, | ||||||
|  | 				BatchLength:  2, | ||||||
|  | 				Workers:      1, | ||||||
|  | 				MaxWorkers:   10, | ||||||
|  | 				QueueLength:  20, | ||||||
|  | 				BlockTimeout: 1 * time.Second, | ||||||
|  | 				BoostTimeout: 5 * time.Minute, | ||||||
|  | 				BoostWorkers: 5, | ||||||
|  | 			}, | ||||||
|  | 		}, &testData{}) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	go queue.Run(func(_ context.Context, shutdown func()) { | ||||||
|  | 		queueShutdown = append(queueShutdown, shutdown) | ||||||
|  | 	}, func(_ context.Context, terminate func()) { | ||||||
|  | 		queueTerminate = append(queueTerminate, terminate) | ||||||
|  | 	}) | ||||||
|  |  | ||||||
|  | 	result3 := <-handleChan | ||||||
|  | 	assert.Equal(t, test1.TestString, result3.TestString) | ||||||
|  | 	assert.Equal(t, test1.TestInt, result3.TestInt) | ||||||
|  |  | ||||||
|  | 	result4 := <-handleChan | ||||||
|  | 	assert.Equal(t, test2.TestString, result4.TestString) | ||||||
|  | 	assert.Equal(t, test2.TestInt, result4.TestInt) | ||||||
|  | 	for _, callback := range queueShutdown { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  | 	for _, callback := range queueTerminate { | ||||||
|  | 		callback() | ||||||
|  | 	} | ||||||
|  | } | ||||||
							
								
								
									
										234
									
								
								modules/queue/queue_redis.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										234
									
								
								modules/queue/queue_redis.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,234 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"encoding/json" | ||||||
|  | 	"errors" | ||||||
|  | 	"fmt" | ||||||
|  | 	"reflect" | ||||||
|  | 	"strings" | ||||||
|  | 	"sync" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  |  | ||||||
|  | 	"github.com/go-redis/redis" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // RedisQueueType is the type for redis queue | ||||||
|  | const RedisQueueType Type = "redis" | ||||||
|  |  | ||||||
|  | type redisClient interface { | ||||||
|  | 	RPush(key string, args ...interface{}) *redis.IntCmd | ||||||
|  | 	LPop(key string) *redis.StringCmd | ||||||
|  | 	Ping() *redis.StatusCmd | ||||||
|  | 	Close() error | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // RedisQueue redis queue | ||||||
|  | type RedisQueue struct { | ||||||
|  | 	pool       *WorkerPool | ||||||
|  | 	client     redisClient | ||||||
|  | 	queueName  string | ||||||
|  | 	closed     chan struct{} | ||||||
|  | 	terminated chan struct{} | ||||||
|  | 	exemplar   interface{} | ||||||
|  | 	workers    int | ||||||
|  | 	name       string | ||||||
|  | 	lock       sync.Mutex | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // RedisQueueConfiguration is the configuration for the redis queue | ||||||
|  | type RedisQueueConfiguration struct { | ||||||
|  | 	Network      string | ||||||
|  | 	Addresses    string | ||||||
|  | 	Password     string | ||||||
|  | 	DBIndex      int | ||||||
|  | 	BatchLength  int | ||||||
|  | 	QueueLength  int | ||||||
|  | 	QueueName    string | ||||||
|  | 	Workers      int | ||||||
|  | 	MaxWorkers   int | ||||||
|  | 	BlockTimeout time.Duration | ||||||
|  | 	BoostTimeout time.Duration | ||||||
|  | 	BoostWorkers int | ||||||
|  | 	Name         string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewRedisQueue creates single redis or cluster redis queue | ||||||
|  | func NewRedisQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | ||||||
|  | 	configInterface, err := toConfig(RedisQueueConfiguration{}, cfg) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	config := configInterface.(RedisQueueConfiguration) | ||||||
|  |  | ||||||
|  | 	dbs := strings.Split(config.Addresses, ",") | ||||||
|  |  | ||||||
|  | 	dataChan := make(chan Data, config.QueueLength) | ||||||
|  | 	ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  |  | ||||||
|  | 	var queue = &RedisQueue{ | ||||||
|  | 		pool: &WorkerPool{ | ||||||
|  | 			baseCtx:            ctx, | ||||||
|  | 			cancel:             cancel, | ||||||
|  | 			batchLength:        config.BatchLength, | ||||||
|  | 			handle:             handle, | ||||||
|  | 			dataChan:           dataChan, | ||||||
|  | 			blockTimeout:       config.BlockTimeout, | ||||||
|  | 			boostTimeout:       config.BoostTimeout, | ||||||
|  | 			boostWorkers:       config.BoostWorkers, | ||||||
|  | 			maxNumberOfWorkers: config.MaxWorkers, | ||||||
|  | 		}, | ||||||
|  | 		queueName: config.QueueName, | ||||||
|  | 		exemplar:  exemplar, | ||||||
|  | 		closed:    make(chan struct{}), | ||||||
|  | 		workers:   config.Workers, | ||||||
|  | 		name:      config.Name, | ||||||
|  | 	} | ||||||
|  | 	if len(dbs) == 0 { | ||||||
|  | 		return nil, errors.New("no redis host specified") | ||||||
|  | 	} else if len(dbs) == 1 { | ||||||
|  | 		queue.client = redis.NewClient(&redis.Options{ | ||||||
|  | 			Network:  config.Network, | ||||||
|  | 			Addr:     strings.TrimSpace(dbs[0]), // use default Addr | ||||||
|  | 			Password: config.Password,           // no password set | ||||||
|  | 			DB:       config.DBIndex,            // use default DB | ||||||
|  | 		}) | ||||||
|  | 	} else { | ||||||
|  | 		queue.client = redis.NewClusterClient(&redis.ClusterOptions{ | ||||||
|  | 			Addrs: dbs, | ||||||
|  | 		}) | ||||||
|  | 	} | ||||||
|  | 	if err := queue.client.Ping().Err(); err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	queue.pool.qid = GetManager().Add(queue, RedisQueueType, config, exemplar, queue.pool) | ||||||
|  |  | ||||||
|  | 	return queue, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run runs the redis queue | ||||||
|  | func (r *RedisQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | ||||||
|  | 	atShutdown(context.Background(), r.Shutdown) | ||||||
|  | 	atTerminate(context.Background(), r.Terminate) | ||||||
|  |  | ||||||
|  | 	go func() { | ||||||
|  | 		_ = r.pool.AddWorkers(r.workers, 0) | ||||||
|  | 	}() | ||||||
|  |  | ||||||
|  | 	go r.readToChan() | ||||||
|  |  | ||||||
|  | 	log.Trace("RedisQueue: %s Waiting til closed", r.name) | ||||||
|  | 	<-r.closed | ||||||
|  | 	log.Trace("RedisQueue: %s Waiting til done", r.name) | ||||||
|  | 	r.pool.Wait() | ||||||
|  |  | ||||||
|  | 	log.Trace("RedisQueue: %s Waiting til cleaned", r.name) | ||||||
|  | 	ctx, cancel := context.WithCancel(context.Background()) | ||||||
|  | 	atTerminate(ctx, cancel) | ||||||
|  | 	r.pool.CleanUp(ctx) | ||||||
|  | 	cancel() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (r *RedisQueue) readToChan() { | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case <-r.closed: | ||||||
|  | 			// tell the pool to shutdown | ||||||
|  | 			r.pool.cancel() | ||||||
|  | 			return | ||||||
|  | 		default: | ||||||
|  | 			bs, err := r.client.LPop(r.queueName).Bytes() | ||||||
|  | 			if err != nil && err != redis.Nil { | ||||||
|  | 				log.Error("RedisQueue: %s Error on LPop: %v", r.name, err) | ||||||
|  | 				time.Sleep(time.Millisecond * 100) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			if len(bs) == 0 { | ||||||
|  | 				time.Sleep(time.Millisecond * 100) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			var data Data | ||||||
|  | 			if r.exemplar != nil { | ||||||
|  | 				t := reflect.TypeOf(r.exemplar) | ||||||
|  | 				n := reflect.New(t) | ||||||
|  | 				ne := n.Elem() | ||||||
|  | 				err = json.Unmarshal(bs, ne.Addr().Interface()) | ||||||
|  | 				data = ne.Interface().(Data) | ||||||
|  | 			} else { | ||||||
|  | 				err = json.Unmarshal(bs, &data) | ||||||
|  | 			} | ||||||
|  | 			if err != nil { | ||||||
|  | 				log.Error("RedisQueue: %s Error on Unmarshal: %v", r.name, err) | ||||||
|  | 				time.Sleep(time.Millisecond * 100) | ||||||
|  | 				continue | ||||||
|  | 			} | ||||||
|  |  | ||||||
|  | 			log.Trace("RedisQueue: %s Task found: %#v", r.name, data) | ||||||
|  | 			r.pool.Push(data) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Push implements Queue | ||||||
|  | func (r *RedisQueue) Push(data Data) error { | ||||||
|  | 	if r.exemplar != nil { | ||||||
|  | 		// Assert data is of same type as r.exemplar | ||||||
|  | 		value := reflect.ValueOf(data) | ||||||
|  | 		t := value.Type() | ||||||
|  | 		exemplarType := reflect.ValueOf(r.exemplar).Type() | ||||||
|  | 		if !t.AssignableTo(exemplarType) || data == nil { | ||||||
|  | 			return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in %s", data, r.exemplar, r.name) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	bs, err := json.Marshal(data) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return err | ||||||
|  | 	} | ||||||
|  | 	return r.client.RPush(r.queueName, bs).Err() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Shutdown processing from this queue | ||||||
|  | func (r *RedisQueue) Shutdown() { | ||||||
|  | 	log.Trace("Shutdown: %s", r.name) | ||||||
|  | 	r.lock.Lock() | ||||||
|  | 	select { | ||||||
|  | 	case <-r.closed: | ||||||
|  | 	default: | ||||||
|  | 		close(r.closed) | ||||||
|  | 	} | ||||||
|  | 	r.lock.Unlock() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Terminate this queue and close the queue | ||||||
|  | func (r *RedisQueue) Terminate() { | ||||||
|  | 	log.Trace("Terminating: %s", r.name) | ||||||
|  | 	r.Shutdown() | ||||||
|  | 	r.lock.Lock() | ||||||
|  | 	select { | ||||||
|  | 	case <-r.terminated: | ||||||
|  | 		r.lock.Unlock() | ||||||
|  | 	default: | ||||||
|  | 		close(r.terminated) | ||||||
|  | 		r.lock.Unlock() | ||||||
|  | 		if err := r.client.Close(); err != nil { | ||||||
|  | 			log.Error("Error whilst closing internal redis client in %s: %v", r.name, err) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Name returns the name of this queue | ||||||
|  | func (r *RedisQueue) Name() string { | ||||||
|  | 	return r.name | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func init() { | ||||||
|  | 	queuesMap[RedisQueueType] = NewRedisQueue | ||||||
|  | } | ||||||
							
								
								
									
										43
									
								
								modules/queue/queue_test.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										43
									
								
								modules/queue/queue_test.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,43 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"encoding/json" | ||||||
|  | 	"testing" | ||||||
|  |  | ||||||
|  | 	"github.com/stretchr/testify/assert" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | type testData struct { | ||||||
|  | 	TestString string | ||||||
|  | 	TestInt    int | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func TestToConfig(t *testing.T) { | ||||||
|  | 	cfg := testData{ | ||||||
|  | 		TestString: "Config", | ||||||
|  | 		TestInt:    10, | ||||||
|  | 	} | ||||||
|  | 	exemplar := testData{} | ||||||
|  |  | ||||||
|  | 	cfg2I, err := toConfig(exemplar, cfg) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	cfg2, ok := (cfg2I).(testData) | ||||||
|  | 	assert.True(t, ok) | ||||||
|  | 	assert.NotEqual(t, cfg2, exemplar) | ||||||
|  | 	assert.Equal(t, &cfg, &cfg2) | ||||||
|  |  | ||||||
|  | 	cfgString, err := json.Marshal(cfg) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  |  | ||||||
|  | 	cfg3I, err := toConfig(exemplar, cfgString) | ||||||
|  | 	assert.NoError(t, err) | ||||||
|  | 	cfg3, ok := (cfg3I).(testData) | ||||||
|  | 	assert.True(t, ok) | ||||||
|  | 	assert.Equal(t, cfg.TestString, cfg3.TestString) | ||||||
|  | 	assert.Equal(t, cfg.TestInt, cfg3.TestInt) | ||||||
|  | 	assert.NotEqual(t, cfg3, exemplar) | ||||||
|  | } | ||||||
							
								
								
									
										206
									
								
								modules/queue/queue_wrapped.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										206
									
								
								modules/queue/queue_wrapped.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,206 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"fmt" | ||||||
|  | 	"reflect" | ||||||
|  | 	"sync" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // WrappedQueueType is the type for a wrapped delayed starting queue | ||||||
|  | const WrappedQueueType Type = "wrapped" | ||||||
|  |  | ||||||
|  | // WrappedQueueConfiguration is the configuration for a WrappedQueue | ||||||
|  | type WrappedQueueConfiguration struct { | ||||||
|  | 	Underlying  Type | ||||||
|  | 	Timeout     time.Duration | ||||||
|  | 	MaxAttempts int | ||||||
|  | 	Config      interface{} | ||||||
|  | 	QueueLength int | ||||||
|  | 	Name        string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | type delayedStarter struct { | ||||||
|  | 	lock        sync.Mutex | ||||||
|  | 	internal    Queue | ||||||
|  | 	underlying  Type | ||||||
|  | 	cfg         interface{} | ||||||
|  | 	timeout     time.Duration | ||||||
|  | 	maxAttempts int | ||||||
|  | 	name        string | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // setInternal must be called with the lock locked. | ||||||
|  | func (q *delayedStarter) setInternal(atShutdown func(context.Context, func()), handle HandlerFunc, exemplar interface{}) error { | ||||||
|  | 	var ctx context.Context | ||||||
|  | 	var cancel context.CancelFunc | ||||||
|  | 	if q.timeout > 0 { | ||||||
|  | 		ctx, cancel = context.WithTimeout(context.Background(), q.timeout) | ||||||
|  | 	} else { | ||||||
|  | 		ctx, cancel = context.WithCancel(context.Background()) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	defer cancel() | ||||||
|  | 	// Ensure we also stop at shutdown | ||||||
|  | 	atShutdown(ctx, func() { | ||||||
|  | 		cancel() | ||||||
|  | 	}) | ||||||
|  |  | ||||||
|  | 	i := 1 | ||||||
|  | 	for q.internal == nil { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			return fmt.Errorf("Timedout creating queue %v with cfg %v in %s", q.underlying, q.cfg, q.name) | ||||||
|  | 		default: | ||||||
|  | 			queue, err := NewQueue(q.underlying, handle, q.cfg, exemplar) | ||||||
|  | 			if err == nil { | ||||||
|  | 				q.internal = queue | ||||||
|  | 				q.lock.Unlock() | ||||||
|  | 				break | ||||||
|  | 			} | ||||||
|  | 			if err.Error() != "resource temporarily unavailable" { | ||||||
|  | 				log.Warn("[Attempt: %d] Failed to create queue: %v for %s cfg: %v error: %v", i, q.underlying, q.name, q.cfg, err) | ||||||
|  | 			} | ||||||
|  | 			i++ | ||||||
|  | 			if q.maxAttempts > 0 && i > q.maxAttempts { | ||||||
|  | 				return fmt.Errorf("Unable to create queue %v for %s with cfg %v by max attempts: error: %v", q.underlying, q.name, q.cfg, err) | ||||||
|  | 			} | ||||||
|  | 			sleepTime := 100 * time.Millisecond | ||||||
|  | 			if q.timeout > 0 && q.maxAttempts > 0 { | ||||||
|  | 				sleepTime = (q.timeout - 200*time.Millisecond) / time.Duration(q.maxAttempts) | ||||||
|  | 			} | ||||||
|  | 			t := time.NewTimer(sleepTime) | ||||||
|  | 			select { | ||||||
|  | 			case <-ctx.Done(): | ||||||
|  | 				t.Stop() | ||||||
|  | 			case <-t.C: | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // WrappedQueue wraps a delayed starting queue | ||||||
|  | type WrappedQueue struct { | ||||||
|  | 	delayedStarter | ||||||
|  | 	handle   HandlerFunc | ||||||
|  | 	exemplar interface{} | ||||||
|  | 	channel  chan Data | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewWrappedQueue will attempt to create a queue of the provided type, | ||||||
|  | // but if there is a problem creating this queue it will instead create | ||||||
|  | // a WrappedQueue with delayed startup of the queue instead and a | ||||||
|  | // channel which will be redirected to the queue | ||||||
|  | func NewWrappedQueue(handle HandlerFunc, cfg, exemplar interface{}) (Queue, error) { | ||||||
|  | 	configInterface, err := toConfig(WrappedQueueConfiguration{}, cfg) | ||||||
|  | 	if err != nil { | ||||||
|  | 		return nil, err | ||||||
|  | 	} | ||||||
|  | 	config := configInterface.(WrappedQueueConfiguration) | ||||||
|  |  | ||||||
|  | 	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} | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	queue = &WrappedQueue{ | ||||||
|  | 		handle:   handle, | ||||||
|  | 		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, | ||||||
|  | 		}, | ||||||
|  | 	} | ||||||
|  | 	_ = GetManager().Add(queue, WrappedQueueType, config, exemplar, nil) | ||||||
|  | 	return queue, nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Name returns the name of the queue | ||||||
|  | func (q *WrappedQueue) Name() string { | ||||||
|  | 	return q.name + "-wrapper" | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Push will push the data to the internal channel checking it against the exemplar | ||||||
|  | func (q *WrappedQueue) Push(data Data) error { | ||||||
|  | 	if q.exemplar != nil { | ||||||
|  | 		// Assert data is of same type as r.exemplar | ||||||
|  | 		value := reflect.ValueOf(data) | ||||||
|  | 		t := value.Type() | ||||||
|  | 		exemplarType := reflect.ValueOf(q.exemplar).Type() | ||||||
|  | 		if !t.AssignableTo(exemplarType) || data == nil { | ||||||
|  | 			return fmt.Errorf("Unable to assign data: %v to same type as exemplar: %v in %s", data, q.exemplar, q.name) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	q.channel <- data | ||||||
|  | 	return nil | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Run starts to run the queue and attempts to create the internal queue | ||||||
|  | func (q *WrappedQueue) Run(atShutdown, atTerminate func(context.Context, func())) { | ||||||
|  | 	q.lock.Lock() | ||||||
|  | 	if q.internal == nil { | ||||||
|  | 		err := q.setInternal(atShutdown, q.handle, q.exemplar) | ||||||
|  | 		q.lock.Unlock() | ||||||
|  | 		if err != nil { | ||||||
|  | 			log.Fatal("Unable to set the internal queue for %s Error: %v", q.Name(), err) | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 		go func() { | ||||||
|  | 			for data := range q.channel { | ||||||
|  | 				_ = q.internal.Push(data) | ||||||
|  | 			} | ||||||
|  | 		}() | ||||||
|  | 	} else { | ||||||
|  | 		q.lock.Unlock() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	q.internal.Run(atShutdown, atTerminate) | ||||||
|  | 	log.Trace("WrappedQueue: %s Done", q.name) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Shutdown this queue and stop processing | ||||||
|  | func (q *WrappedQueue) Shutdown() { | ||||||
|  | 	log.Trace("WrappedQueue: %s Shutdown", q.name) | ||||||
|  | 	q.lock.Lock() | ||||||
|  | 	defer q.lock.Unlock() | ||||||
|  | 	if q.internal == nil { | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	if shutdownable, ok := q.internal.(Shutdownable); ok { | ||||||
|  | 		shutdownable.Shutdown() | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Terminate this queue and close the queue | ||||||
|  | func (q *WrappedQueue) Terminate() { | ||||||
|  | 	log.Trace("WrappedQueue: %s Terminating", q.name) | ||||||
|  | 	q.lock.Lock() | ||||||
|  | 	defer q.lock.Unlock() | ||||||
|  | 	if q.internal == nil { | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	if shutdownable, ok := q.internal.(Shutdownable); ok { | ||||||
|  | 		shutdownable.Terminate() | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func init() { | ||||||
|  | 	queuesMap[WrappedQueueType] = NewWrappedQueue | ||||||
|  | } | ||||||
							
								
								
									
										75
									
								
								modules/queue/setting.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										75
									
								
								modules/queue/setting.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,75 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"encoding/json" | ||||||
|  | 	"fmt" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | 	"code.gitea.io/gitea/modules/setting" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | func validType(t string) (Type, error) { | ||||||
|  | 	if len(t) == 0 { | ||||||
|  | 		return PersistableChannelQueueType, nil | ||||||
|  | 	} | ||||||
|  | 	for _, typ := range RegisteredTypes() { | ||||||
|  | 		if t == string(typ) { | ||||||
|  | 			return typ, nil | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return PersistableChannelQueueType, fmt.Errorf("Unknown queue type: %s defaulting to %s", t, string(PersistableChannelQueueType)) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // CreateQueue for name with provided handler and exemplar | ||||||
|  | func CreateQueue(name string, handle HandlerFunc, exemplar interface{}) Queue { | ||||||
|  | 	q := setting.GetQueueSettings(name) | ||||||
|  | 	opts := make(map[string]interface{}) | ||||||
|  | 	opts["Name"] = name | ||||||
|  | 	opts["QueueLength"] = q.Length | ||||||
|  | 	opts["BatchLength"] = q.BatchLength | ||||||
|  | 	opts["DataDir"] = q.DataDir | ||||||
|  | 	opts["Addresses"] = q.Addresses | ||||||
|  | 	opts["Network"] = q.Network | ||||||
|  | 	opts["Password"] = q.Password | ||||||
|  | 	opts["DBIndex"] = q.DBIndex | ||||||
|  | 	opts["QueueName"] = q.QueueName | ||||||
|  | 	opts["Workers"] = q.Workers | ||||||
|  | 	opts["MaxWorkers"] = q.MaxWorkers | ||||||
|  | 	opts["BlockTimeout"] = q.BlockTimeout | ||||||
|  | 	opts["BoostTimeout"] = q.BoostTimeout | ||||||
|  | 	opts["BoostWorkers"] = q.BoostWorkers | ||||||
|  |  | ||||||
|  | 	typ, err := validType(q.Type) | ||||||
|  | 	if err != nil { | ||||||
|  | 		log.Error("Invalid type %s provided for queue named %s defaulting to %s", q.Type, name, string(typ)) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	cfg, err := json.Marshal(opts) | ||||||
|  | 	if err != nil { | ||||||
|  | 		log.Error("Unable to marshall generic options: %v Error: %v", opts, err) | ||||||
|  | 		log.Error("Unable to create queue for %s", name, err) | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	returnable, err := NewQueue(typ, handle, cfg, exemplar) | ||||||
|  | 	if q.WrapIfNecessary && err != nil { | ||||||
|  | 		log.Warn("Unable to create queue for %s: %v", name, err) | ||||||
|  | 		log.Warn("Attempting to create wrapped queue") | ||||||
|  | 		returnable, err = NewQueue(WrappedQueueType, handle, WrappedQueueConfiguration{ | ||||||
|  | 			Underlying:  Type(q.Type), | ||||||
|  | 			Timeout:     q.Timeout, | ||||||
|  | 			MaxAttempts: q.MaxAttempts, | ||||||
|  | 			Config:      cfg, | ||||||
|  | 			QueueLength: q.Length, | ||||||
|  | 		}, exemplar) | ||||||
|  | 	} | ||||||
|  | 	if err != nil { | ||||||
|  | 		log.Error("Unable to create queue for %s: %v", name, err) | ||||||
|  | 		return nil | ||||||
|  | 	} | ||||||
|  | 	return returnable | ||||||
|  | } | ||||||
							
								
								
									
										325
									
								
								modules/queue/workerpool.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										325
									
								
								modules/queue/workerpool.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,325 @@ | |||||||
|  | // 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 ( | ||||||
|  | 	"context" | ||||||
|  | 	"sync" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // WorkerPool takes | ||||||
|  | type WorkerPool struct { | ||||||
|  | 	lock               sync.Mutex | ||||||
|  | 	baseCtx            context.Context | ||||||
|  | 	cancel             context.CancelFunc | ||||||
|  | 	cond               *sync.Cond | ||||||
|  | 	qid                int64 | ||||||
|  | 	maxNumberOfWorkers int | ||||||
|  | 	numberOfWorkers    int | ||||||
|  | 	batchLength        int | ||||||
|  | 	handle             HandlerFunc | ||||||
|  | 	dataChan           chan Data | ||||||
|  | 	blockTimeout       time.Duration | ||||||
|  | 	boostTimeout       time.Duration | ||||||
|  | 	boostWorkers       int | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Push pushes the data to the internal channel | ||||||
|  | func (p *WorkerPool) Push(data Data) { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	if p.blockTimeout > 0 && p.boostTimeout > 0 && (p.numberOfWorkers <= p.maxNumberOfWorkers || p.maxNumberOfWorkers < 0) { | ||||||
|  | 		p.lock.Unlock() | ||||||
|  | 		p.pushBoost(data) | ||||||
|  | 	} else { | ||||||
|  | 		p.lock.Unlock() | ||||||
|  | 		p.dataChan <- data | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (p *WorkerPool) pushBoost(data Data) { | ||||||
|  | 	select { | ||||||
|  | 	case p.dataChan <- data: | ||||||
|  | 	default: | ||||||
|  | 		p.lock.Lock() | ||||||
|  | 		if p.blockTimeout <= 0 { | ||||||
|  | 			p.lock.Unlock() | ||||||
|  | 			p.dataChan <- data | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 		ourTimeout := p.blockTimeout | ||||||
|  | 		timer := time.NewTimer(p.blockTimeout) | ||||||
|  | 		p.lock.Unlock() | ||||||
|  | 		select { | ||||||
|  | 		case p.dataChan <- data: | ||||||
|  | 			if timer.Stop() { | ||||||
|  | 				select { | ||||||
|  | 				case <-timer.C: | ||||||
|  | 				default: | ||||||
|  | 				} | ||||||
|  | 			} | ||||||
|  | 		case <-timer.C: | ||||||
|  | 			p.lock.Lock() | ||||||
|  | 			if p.blockTimeout > ourTimeout || (p.numberOfWorkers > p.maxNumberOfWorkers && p.maxNumberOfWorkers >= 0) { | ||||||
|  | 				p.lock.Unlock() | ||||||
|  | 				p.dataChan <- data | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 			p.blockTimeout *= 2 | ||||||
|  | 			ctx, cancel := context.WithCancel(p.baseCtx) | ||||||
|  | 			mq := GetManager().GetManagedQueue(p.qid) | ||||||
|  | 			boost := p.boostWorkers | ||||||
|  | 			if (boost+p.numberOfWorkers) > p.maxNumberOfWorkers && p.maxNumberOfWorkers >= 0 { | ||||||
|  | 				boost = p.maxNumberOfWorkers - p.numberOfWorkers | ||||||
|  | 			} | ||||||
|  | 			if mq != nil { | ||||||
|  | 				log.Warn("WorkerPool: %d (for %s) Channel blocked for %v - adding %d temporary workers for %s, block timeout now %v", p.qid, mq.Name, ourTimeout, boost, p.boostTimeout, p.blockTimeout) | ||||||
|  |  | ||||||
|  | 				start := time.Now() | ||||||
|  | 				pid := mq.RegisterWorkers(boost, start, false, start, cancel) | ||||||
|  | 				go func() { | ||||||
|  | 					<-ctx.Done() | ||||||
|  | 					mq.RemoveWorkers(pid) | ||||||
|  | 					cancel() | ||||||
|  | 				}() | ||||||
|  | 			} else { | ||||||
|  | 				log.Warn("WorkerPool: %d Channel blocked for %v - adding %d temporary workers for %s, block timeout now %v", p.qid, ourTimeout, p.boostWorkers, p.boostTimeout, p.blockTimeout) | ||||||
|  | 			} | ||||||
|  | 			go func() { | ||||||
|  | 				<-time.After(p.boostTimeout) | ||||||
|  | 				cancel() | ||||||
|  | 				p.lock.Lock() | ||||||
|  | 				p.blockTimeout /= 2 | ||||||
|  | 				p.lock.Unlock() | ||||||
|  | 			}() | ||||||
|  | 			p.addWorkers(ctx, boost) | ||||||
|  | 			p.lock.Unlock() | ||||||
|  | 			p.dataChan <- data | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NumberOfWorkers returns the number of current workers in the pool | ||||||
|  | func (p *WorkerPool) NumberOfWorkers() int { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	return p.numberOfWorkers | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // MaxNumberOfWorkers returns the maximum number of workers automatically added to the pool | ||||||
|  | func (p *WorkerPool) MaxNumberOfWorkers() int { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	return p.maxNumberOfWorkers | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // BoostWorkers returns the number of workers for a boost | ||||||
|  | func (p *WorkerPool) BoostWorkers() int { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	return p.boostWorkers | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // BoostTimeout returns the timeout of the next boost | ||||||
|  | func (p *WorkerPool) BoostTimeout() time.Duration { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	return p.boostTimeout | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // BlockTimeout returns the timeout til the next boost | ||||||
|  | func (p *WorkerPool) BlockTimeout() time.Duration { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	return p.blockTimeout | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // SetSettings sets the setable boost values | ||||||
|  | func (p *WorkerPool) SetSettings(maxNumberOfWorkers, boostWorkers int, timeout time.Duration) { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	p.maxNumberOfWorkers = maxNumberOfWorkers | ||||||
|  | 	p.boostWorkers = boostWorkers | ||||||
|  | 	p.boostTimeout = timeout | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // SetMaxNumberOfWorkers sets the maximum number of workers automatically added to the pool | ||||||
|  | // Changing this number will not change the number of current workers but will change the limit | ||||||
|  | // for future additions | ||||||
|  | func (p *WorkerPool) SetMaxNumberOfWorkers(newMax int) { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	p.maxNumberOfWorkers = newMax | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // AddWorkers adds workers to the pool - this allows the number of workers to go above the limit | ||||||
|  | func (p *WorkerPool) AddWorkers(number int, timeout time.Duration) context.CancelFunc { | ||||||
|  | 	var ctx context.Context | ||||||
|  | 	var cancel context.CancelFunc | ||||||
|  | 	start := time.Now() | ||||||
|  | 	end := start | ||||||
|  | 	hasTimeout := false | ||||||
|  | 	if timeout > 0 { | ||||||
|  | 		ctx, cancel = context.WithTimeout(p.baseCtx, timeout) | ||||||
|  | 		end = start.Add(timeout) | ||||||
|  | 		hasTimeout = true | ||||||
|  | 	} else { | ||||||
|  | 		ctx, cancel = context.WithCancel(p.baseCtx) | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	mq := GetManager().GetManagedQueue(p.qid) | ||||||
|  | 	if mq != nil { | ||||||
|  | 		pid := mq.RegisterWorkers(number, start, hasTimeout, end, cancel) | ||||||
|  | 		go func() { | ||||||
|  | 			<-ctx.Done() | ||||||
|  | 			mq.RemoveWorkers(pid) | ||||||
|  | 			cancel() | ||||||
|  | 		}() | ||||||
|  | 		log.Trace("WorkerPool: %d (for %s) adding %d workers with group id: %d", p.qid, mq.Name, number, pid) | ||||||
|  | 	} else { | ||||||
|  | 		log.Trace("WorkerPool: %d adding %d workers (no group id)", p.qid, number) | ||||||
|  |  | ||||||
|  | 	} | ||||||
|  | 	p.addWorkers(ctx, number) | ||||||
|  | 	return cancel | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // addWorkers adds workers to the pool | ||||||
|  | func (p *WorkerPool) addWorkers(ctx context.Context, number int) { | ||||||
|  | 	for i := 0; i < number; i++ { | ||||||
|  | 		p.lock.Lock() | ||||||
|  | 		if p.cond == nil { | ||||||
|  | 			p.cond = sync.NewCond(&p.lock) | ||||||
|  | 		} | ||||||
|  | 		p.numberOfWorkers++ | ||||||
|  | 		p.lock.Unlock() | ||||||
|  | 		go func() { | ||||||
|  | 			p.doWork(ctx) | ||||||
|  |  | ||||||
|  | 			p.lock.Lock() | ||||||
|  | 			p.numberOfWorkers-- | ||||||
|  | 			if p.numberOfWorkers == 0 { | ||||||
|  | 				p.cond.Broadcast() | ||||||
|  | 			} else if p.numberOfWorkers < 0 { | ||||||
|  | 				// numberOfWorkers can't go negative but... | ||||||
|  | 				log.Warn("Number of Workers < 0 for QID %d - this shouldn't happen", p.qid) | ||||||
|  | 				p.numberOfWorkers = 0 | ||||||
|  | 				p.cond.Broadcast() | ||||||
|  | 			} | ||||||
|  | 			p.lock.Unlock() | ||||||
|  | 		}() | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Wait for WorkerPool to finish | ||||||
|  | func (p *WorkerPool) Wait() { | ||||||
|  | 	p.lock.Lock() | ||||||
|  | 	defer p.lock.Unlock() | ||||||
|  | 	if p.cond == nil { | ||||||
|  | 		p.cond = sync.NewCond(&p.lock) | ||||||
|  | 	} | ||||||
|  | 	if p.numberOfWorkers <= 0 { | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	p.cond.Wait() | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // CleanUp will drain the remaining contents of the channel | ||||||
|  | // This should be called after AddWorkers context is closed | ||||||
|  | func (p *WorkerPool) CleanUp(ctx context.Context) { | ||||||
|  | 	log.Trace("WorkerPool: %d CleanUp", p.qid) | ||||||
|  | 	close(p.dataChan) | ||||||
|  | 	for data := range p.dataChan { | ||||||
|  | 		p.handle(data) | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			log.Warn("WorkerPool: %d Cleanup context closed before finishing clean-up", p.qid) | ||||||
|  | 			return | ||||||
|  | 		default: | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	log.Trace("WorkerPool: %d CleanUp Done", p.qid) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | func (p *WorkerPool) doWork(ctx context.Context) { | ||||||
|  | 	delay := time.Millisecond * 300 | ||||||
|  | 	var data = make([]Data, 0, p.batchLength) | ||||||
|  | 	for { | ||||||
|  | 		select { | ||||||
|  | 		case <-ctx.Done(): | ||||||
|  | 			if len(data) > 0 { | ||||||
|  | 				log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 				p.handle(data...) | ||||||
|  | 			} | ||||||
|  | 			log.Trace("Worker shutting down") | ||||||
|  | 			return | ||||||
|  | 		case datum, ok := <-p.dataChan: | ||||||
|  | 			if !ok { | ||||||
|  | 				// the dataChan has been closed - we should finish up: | ||||||
|  | 				if len(data) > 0 { | ||||||
|  | 					log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 					p.handle(data...) | ||||||
|  | 				} | ||||||
|  | 				log.Trace("Worker shutting down") | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 			data = append(data, datum) | ||||||
|  | 			if len(data) >= p.batchLength { | ||||||
|  | 				log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 				p.handle(data...) | ||||||
|  | 				data = make([]Data, 0, p.batchLength) | ||||||
|  | 			} | ||||||
|  | 		default: | ||||||
|  | 			timer := time.NewTimer(delay) | ||||||
|  | 			select { | ||||||
|  | 			case <-ctx.Done(): | ||||||
|  | 				if timer.Stop() { | ||||||
|  | 					select { | ||||||
|  | 					case <-timer.C: | ||||||
|  | 					default: | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				if len(data) > 0 { | ||||||
|  | 					log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 					p.handle(data...) | ||||||
|  | 				} | ||||||
|  | 				log.Trace("Worker shutting down") | ||||||
|  | 				return | ||||||
|  | 			case datum, ok := <-p.dataChan: | ||||||
|  | 				if timer.Stop() { | ||||||
|  | 					select { | ||||||
|  | 					case <-timer.C: | ||||||
|  | 					default: | ||||||
|  | 					} | ||||||
|  | 				} | ||||||
|  | 				if !ok { | ||||||
|  | 					// the dataChan has been closed - we should finish up: | ||||||
|  | 					if len(data) > 0 { | ||||||
|  | 						log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 						p.handle(data...) | ||||||
|  | 					} | ||||||
|  | 					log.Trace("Worker shutting down") | ||||||
|  | 					return | ||||||
|  | 				} | ||||||
|  | 				data = append(data, datum) | ||||||
|  | 				if len(data) >= p.batchLength { | ||||||
|  | 					log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 					p.handle(data...) | ||||||
|  | 					data = make([]Data, 0, p.batchLength) | ||||||
|  | 				} | ||||||
|  | 			case <-timer.C: | ||||||
|  | 				delay = time.Millisecond * 100 | ||||||
|  | 				if len(data) > 0 { | ||||||
|  | 					log.Trace("Handling: %d data, %v", len(data), data) | ||||||
|  | 					p.handle(data...) | ||||||
|  | 					data = make([]Data, 0, p.batchLength) | ||||||
|  | 				} | ||||||
|  |  | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
							
								
								
									
										159
									
								
								modules/setting/queue.go
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										159
									
								
								modules/setting/queue.go
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,159 @@ | |||||||
|  | // 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 setting | ||||||
|  |  | ||||||
|  | import ( | ||||||
|  | 	"fmt" | ||||||
|  | 	"path" | ||||||
|  | 	"strconv" | ||||||
|  | 	"strings" | ||||||
|  | 	"time" | ||||||
|  |  | ||||||
|  | 	"code.gitea.io/gitea/modules/log" | ||||||
|  | ) | ||||||
|  |  | ||||||
|  | // QueueSettings represent the settings for a queue from the ini | ||||||
|  | type QueueSettings struct { | ||||||
|  | 	DataDir          string | ||||||
|  | 	Length           int | ||||||
|  | 	BatchLength      int | ||||||
|  | 	ConnectionString string | ||||||
|  | 	Type             string | ||||||
|  | 	Network          string | ||||||
|  | 	Addresses        string | ||||||
|  | 	Password         string | ||||||
|  | 	QueueName        string | ||||||
|  | 	DBIndex          int | ||||||
|  | 	WrapIfNecessary  bool | ||||||
|  | 	MaxAttempts      int | ||||||
|  | 	Timeout          time.Duration | ||||||
|  | 	Workers          int | ||||||
|  | 	MaxWorkers       int | ||||||
|  | 	BlockTimeout     time.Duration | ||||||
|  | 	BoostTimeout     time.Duration | ||||||
|  | 	BoostWorkers     int | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // Queue settings | ||||||
|  | var Queue = QueueSettings{} | ||||||
|  |  | ||||||
|  | // GetQueueSettings returns the queue settings for the appropriately named queue | ||||||
|  | func GetQueueSettings(name string) QueueSettings { | ||||||
|  | 	q := QueueSettings{} | ||||||
|  | 	sec := Cfg.Section("queue." + name) | ||||||
|  | 	// DataDir is not directly inheritable | ||||||
|  | 	q.DataDir = path.Join(Queue.DataDir, name) | ||||||
|  | 	// QueueName is not directly inheritable either | ||||||
|  | 	q.QueueName = name + Queue.QueueName | ||||||
|  | 	for _, key := range sec.Keys() { | ||||||
|  | 		switch key.Name() { | ||||||
|  | 		case "DATADIR": | ||||||
|  | 			q.DataDir = key.MustString(q.DataDir) | ||||||
|  | 		case "QUEUE_NAME": | ||||||
|  | 			q.QueueName = key.MustString(q.QueueName) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	if !path.IsAbs(q.DataDir) { | ||||||
|  | 		q.DataDir = path.Join(AppDataPath, q.DataDir) | ||||||
|  | 	} | ||||||
|  | 	sec.Key("DATADIR").SetValue(q.DataDir) | ||||||
|  | 	// The rest are... | ||||||
|  | 	q.Length = sec.Key("LENGTH").MustInt(Queue.Length) | ||||||
|  | 	q.BatchLength = sec.Key("BATCH_LENGTH").MustInt(Queue.BatchLength) | ||||||
|  | 	q.ConnectionString = sec.Key("CONN_STR").MustString(Queue.ConnectionString) | ||||||
|  | 	q.Type = sec.Key("TYPE").MustString(Queue.Type) | ||||||
|  | 	q.WrapIfNecessary = sec.Key("WRAP_IF_NECESSARY").MustBool(Queue.WrapIfNecessary) | ||||||
|  | 	q.MaxAttempts = sec.Key("MAX_ATTEMPTS").MustInt(Queue.MaxAttempts) | ||||||
|  | 	q.Timeout = sec.Key("TIMEOUT").MustDuration(Queue.Timeout) | ||||||
|  | 	q.Workers = sec.Key("WORKERS").MustInt(Queue.Workers) | ||||||
|  | 	q.MaxWorkers = sec.Key("MAX_WORKERS").MustInt(Queue.MaxWorkers) | ||||||
|  | 	q.BlockTimeout = sec.Key("BLOCK_TIMEOUT").MustDuration(Queue.BlockTimeout) | ||||||
|  | 	q.BoostTimeout = sec.Key("BOOST_TIMEOUT").MustDuration(Queue.BoostTimeout) | ||||||
|  | 	q.BoostWorkers = sec.Key("BOOST_WORKERS").MustInt(Queue.BoostWorkers) | ||||||
|  |  | ||||||
|  | 	q.Network, q.Addresses, q.Password, q.DBIndex, _ = ParseQueueConnStr(q.ConnectionString) | ||||||
|  | 	return q | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // NewQueueService sets up the default settings for Queues | ||||||
|  | // This is exported for tests to be able to use the queue | ||||||
|  | func NewQueueService() { | ||||||
|  | 	sec := Cfg.Section("queue") | ||||||
|  | 	Queue.DataDir = sec.Key("DATADIR").MustString("queues/") | ||||||
|  | 	if !path.IsAbs(Queue.DataDir) { | ||||||
|  | 		Queue.DataDir = path.Join(AppDataPath, Queue.DataDir) | ||||||
|  | 	} | ||||||
|  | 	Queue.Length = sec.Key("LENGTH").MustInt(20) | ||||||
|  | 	Queue.BatchLength = sec.Key("BATCH_LENGTH").MustInt(20) | ||||||
|  | 	Queue.ConnectionString = sec.Key("CONN_STR").MustString(path.Join(AppDataPath, "")) | ||||||
|  | 	Queue.Type = sec.Key("TYPE").MustString("") | ||||||
|  | 	Queue.Network, Queue.Addresses, Queue.Password, Queue.DBIndex, _ = ParseQueueConnStr(Queue.ConnectionString) | ||||||
|  | 	Queue.WrapIfNecessary = sec.Key("WRAP_IF_NECESSARY").MustBool(true) | ||||||
|  | 	Queue.MaxAttempts = sec.Key("MAX_ATTEMPTS").MustInt(10) | ||||||
|  | 	Queue.Timeout = sec.Key("TIMEOUT").MustDuration(GracefulHammerTime + 30*time.Second) | ||||||
|  | 	Queue.Workers = sec.Key("WORKERS").MustInt(1) | ||||||
|  | 	Queue.MaxWorkers = sec.Key("MAX_WORKERS").MustInt(10) | ||||||
|  | 	Queue.BlockTimeout = sec.Key("BLOCK_TIMEOUT").MustDuration(1 * time.Second) | ||||||
|  | 	Queue.BoostTimeout = sec.Key("BOOST_TIMEOUT").MustDuration(5 * time.Minute) | ||||||
|  | 	Queue.BoostWorkers = sec.Key("BOOST_WORKERS").MustInt(5) | ||||||
|  | 	Queue.QueueName = sec.Key("QUEUE_NAME").MustString("_queue") | ||||||
|  |  | ||||||
|  | 	// Now handle the old issue_indexer configuration | ||||||
|  | 	section := Cfg.Section("queue.issue_indexer") | ||||||
|  | 	issueIndexerSectionMap := map[string]string{} | ||||||
|  | 	for _, key := range section.Keys() { | ||||||
|  | 		issueIndexerSectionMap[key.Name()] = key.Value() | ||||||
|  | 	} | ||||||
|  | 	if _, ok := issueIndexerSectionMap["TYPE"]; !ok { | ||||||
|  | 		switch Indexer.IssueQueueType { | ||||||
|  | 		case LevelQueueType: | ||||||
|  | 			section.Key("TYPE").SetValue("level") | ||||||
|  | 		case ChannelQueueType: | ||||||
|  | 			section.Key("TYPE").SetValue("persistable-channel") | ||||||
|  | 		case RedisQueueType: | ||||||
|  | 			section.Key("TYPE").SetValue("redis") | ||||||
|  | 		default: | ||||||
|  | 			log.Fatal("Unsupported indexer queue type: %v", | ||||||
|  | 				Indexer.IssueQueueType) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	if _, ok := issueIndexerSectionMap["LENGTH"]; !ok { | ||||||
|  | 		section.Key("LENGTH").SetValue(fmt.Sprintf("%d", Indexer.UpdateQueueLength)) | ||||||
|  | 	} | ||||||
|  | 	if _, ok := issueIndexerSectionMap["BATCH_LENGTH"]; !ok { | ||||||
|  | 		section.Key("BATCH_LENGTH").SetValue(fmt.Sprintf("%d", Indexer.IssueQueueBatchNumber)) | ||||||
|  | 	} | ||||||
|  | 	if _, ok := issueIndexerSectionMap["DATADIR"]; !ok { | ||||||
|  | 		section.Key("DATADIR").SetValue(Indexer.IssueQueueDir) | ||||||
|  | 	} | ||||||
|  | 	if _, ok := issueIndexerSectionMap["CONN_STR"]; !ok { | ||||||
|  | 		section.Key("CONN_STR").SetValue(Indexer.IssueQueueConnStr) | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // ParseQueueConnStr parses a queue connection string | ||||||
|  | func ParseQueueConnStr(connStr string) (network, addrs, password string, dbIdx int, err error) { | ||||||
|  | 	fields := strings.Fields(connStr) | ||||||
|  | 	for _, f := range fields { | ||||||
|  | 		items := strings.SplitN(f, "=", 2) | ||||||
|  | 		if len(items) < 2 { | ||||||
|  | 			continue | ||||||
|  | 		} | ||||||
|  | 		switch strings.ToLower(items[0]) { | ||||||
|  | 		case "network": | ||||||
|  | 			network = items[1] | ||||||
|  | 		case "addrs": | ||||||
|  | 			addrs = items[1] | ||||||
|  | 		case "password": | ||||||
|  | 			password = items[1] | ||||||
|  | 		case "db": | ||||||
|  | 			dbIdx, err = strconv.Atoi(items[1]) | ||||||
|  | 			if err != nil { | ||||||
|  | 				return | ||||||
|  | 			} | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | 	return | ||||||
|  | } | ||||||
| @@ -1093,4 +1093,5 @@ func NewServices() { | |||||||
| 	newMigrationsService() | 	newMigrationsService() | ||||||
| 	newIndexerService() | 	newIndexerService() | ||||||
| 	newTaskService() | 	newTaskService() | ||||||
|  | 	NewQueueService() | ||||||
| } | } | ||||||
|   | |||||||
| @@ -4,22 +4,15 @@ | |||||||
|  |  | ||||||
| package setting | package setting | ||||||
|  |  | ||||||
| var ( |  | ||||||
| 	// Task settings |  | ||||||
| 	Task = struct { |  | ||||||
| 		QueueType    string |  | ||||||
| 		QueueLength  int |  | ||||||
| 		QueueConnStr string |  | ||||||
| 	}{ |  | ||||||
| 		QueueType:    ChannelQueueType, |  | ||||||
| 		QueueLength:  1000, |  | ||||||
| 		QueueConnStr: "addrs=127.0.0.1:6379 db=0", |  | ||||||
| 	} |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| func newTaskService() { | func newTaskService() { | ||||||
| 	sec := Cfg.Section("task") | 	taskSec := Cfg.Section("task") | ||||||
| 	Task.QueueType = sec.Key("QUEUE_TYPE").MustString(ChannelQueueType) | 	queueTaskSec := Cfg.Section("queue.task") | ||||||
| 	Task.QueueLength = sec.Key("QUEUE_LENGTH").MustInt(1000) | 	switch taskSec.Key("QUEUE_TYPE").MustString(ChannelQueueType) { | ||||||
| 	Task.QueueConnStr = sec.Key("QUEUE_CONN_STR").MustString("addrs=127.0.0.1:6379 db=0") | 	case ChannelQueueType: | ||||||
|  | 		queueTaskSec.Key("TYPE").MustString("persistable-channel") | ||||||
|  | 	case RedisQueueType: | ||||||
|  | 		queueTaskSec.Key("TYPE").MustString("redis") | ||||||
|  | 	} | ||||||
|  | 	queueTaskSec.Key("LENGTH").MustInt(taskSec.Key("QUEUE_LENGTH").MustInt(1000)) | ||||||
|  | 	queueTaskSec.Key("CONN_STR").MustString(taskSec.Key("QUEUE_CONN_STR").MustString("addrs=127.0.0.1:6379 db=0")) | ||||||
| } | } | ||||||
|   | |||||||
| @@ -1,14 +0,0 @@ | |||||||
| // Copyright 2019 Gitea. All rights reserved. |  | ||||||
| // Use of this source code is governed by a MIT-style |  | ||||||
| // license that can be found in the LICENSE file. |  | ||||||
|  |  | ||||||
| package task |  | ||||||
|  |  | ||||||
| import "code.gitea.io/gitea/models" |  | ||||||
|  |  | ||||||
| // Queue defines an interface to run task queue |  | ||||||
| type Queue interface { |  | ||||||
| 	Run() error |  | ||||||
| 	Push(*models.Task) error |  | ||||||
| 	Stop() |  | ||||||
| } |  | ||||||
| @@ -1,48 +0,0 @@ | |||||||
| // 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 task |  | ||||||
|  |  | ||||||
| import ( |  | ||||||
| 	"code.gitea.io/gitea/models" |  | ||||||
| 	"code.gitea.io/gitea/modules/log" |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| var ( |  | ||||||
| 	_ Queue = &ChannelQueue{} |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| // ChannelQueue implements |  | ||||||
| type ChannelQueue struct { |  | ||||||
| 	queue chan *models.Task |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // NewChannelQueue create a memory channel queue |  | ||||||
| func NewChannelQueue(queueLen int) *ChannelQueue { |  | ||||||
| 	return &ChannelQueue{ |  | ||||||
| 		queue: make(chan *models.Task, queueLen), |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Run starts to run the queue |  | ||||||
| func (c *ChannelQueue) Run() error { |  | ||||||
| 	for task := range c.queue { |  | ||||||
| 		err := Run(task) |  | ||||||
| 		if err != nil { |  | ||||||
| 			log.Error("Run task failed: %s", err.Error()) |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Push will push the task ID to queue |  | ||||||
| func (c *ChannelQueue) Push(task *models.Task) error { |  | ||||||
| 	c.queue <- task |  | ||||||
| 	return nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Stop stop the queue |  | ||||||
| func (c *ChannelQueue) Stop() { |  | ||||||
| 	close(c.queue) |  | ||||||
| } |  | ||||||
| @@ -1,130 +0,0 @@ | |||||||
| // 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 task |  | ||||||
|  |  | ||||||
| import ( |  | ||||||
| 	"encoding/json" |  | ||||||
| 	"errors" |  | ||||||
| 	"strconv" |  | ||||||
| 	"strings" |  | ||||||
| 	"time" |  | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/models" |  | ||||||
| 	"code.gitea.io/gitea/modules/log" |  | ||||||
|  |  | ||||||
| 	"github.com/go-redis/redis" |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| var ( |  | ||||||
| 	_ Queue = &RedisQueue{} |  | ||||||
| ) |  | ||||||
|  |  | ||||||
| type redisClient interface { |  | ||||||
| 	RPush(key string, args ...interface{}) *redis.IntCmd |  | ||||||
| 	LPop(key string) *redis.StringCmd |  | ||||||
| 	Ping() *redis.StatusCmd |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // RedisQueue redis queue |  | ||||||
| type RedisQueue struct { |  | ||||||
| 	client    redisClient |  | ||||||
| 	queueName string |  | ||||||
| 	closeChan chan bool |  | ||||||
| } |  | ||||||
|  |  | ||||||
| func parseConnStr(connStr string) (addrs, password string, dbIdx int, err error) { |  | ||||||
| 	fields := strings.Fields(connStr) |  | ||||||
| 	for _, f := range fields { |  | ||||||
| 		items := strings.SplitN(f, "=", 2) |  | ||||||
| 		if len(items) < 2 { |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
| 		switch strings.ToLower(items[0]) { |  | ||||||
| 		case "addrs": |  | ||||||
| 			addrs = items[1] |  | ||||||
| 		case "password": |  | ||||||
| 			password = items[1] |  | ||||||
| 		case "db": |  | ||||||
| 			dbIdx, err = strconv.Atoi(items[1]) |  | ||||||
| 			if err != nil { |  | ||||||
| 				return |  | ||||||
| 			} |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| 	return |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // NewRedisQueue creates single redis or cluster redis queue |  | ||||||
| func NewRedisQueue(addrs string, password string, dbIdx int) (*RedisQueue, error) { |  | ||||||
| 	dbs := strings.Split(addrs, ",") |  | ||||||
| 	var queue = RedisQueue{ |  | ||||||
| 		queueName: "task_queue", |  | ||||||
| 		closeChan: make(chan bool), |  | ||||||
| 	} |  | ||||||
| 	if len(dbs) == 0 { |  | ||||||
| 		return nil, errors.New("no redis host found") |  | ||||||
| 	} else if len(dbs) == 1 { |  | ||||||
| 		queue.client = redis.NewClient(&redis.Options{ |  | ||||||
| 			Addr:     strings.TrimSpace(dbs[0]), // use default Addr |  | ||||||
| 			Password: password,                  // no password set |  | ||||||
| 			DB:       dbIdx,                     // use default DB |  | ||||||
| 		}) |  | ||||||
| 	} else { |  | ||||||
| 		// cluster will ignore db |  | ||||||
| 		queue.client = redis.NewClusterClient(&redis.ClusterOptions{ |  | ||||||
| 			Addrs:    dbs, |  | ||||||
| 			Password: password, |  | ||||||
| 		}) |  | ||||||
| 	} |  | ||||||
| 	if err := queue.client.Ping().Err(); err != nil { |  | ||||||
| 		return nil, err |  | ||||||
| 	} |  | ||||||
| 	return &queue, nil |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Run starts to run the queue |  | ||||||
| func (r *RedisQueue) Run() error { |  | ||||||
| 	for { |  | ||||||
| 		select { |  | ||||||
| 		case <-r.closeChan: |  | ||||||
| 			return nil |  | ||||||
| 		case <-time.After(time.Millisecond * 100): |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		bs, err := r.client.LPop(r.queueName).Bytes() |  | ||||||
| 		if err != nil { |  | ||||||
| 			if err != redis.Nil { |  | ||||||
| 				log.Error("LPop failed: %v", err) |  | ||||||
| 			} |  | ||||||
| 			time.Sleep(time.Millisecond * 100) |  | ||||||
| 			continue |  | ||||||
| 		} |  | ||||||
|  |  | ||||||
| 		var task models.Task |  | ||||||
| 		err = json.Unmarshal(bs, &task) |  | ||||||
| 		if err != nil { |  | ||||||
| 			log.Error("Unmarshal task failed: %s", err.Error()) |  | ||||||
| 		} else { |  | ||||||
| 			err = Run(&task) |  | ||||||
| 			if err != nil { |  | ||||||
| 				log.Error("Run task failed: %s", err.Error()) |  | ||||||
| 			} |  | ||||||
| 		} |  | ||||||
| 	} |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Push implements Queue |  | ||||||
| func (r *RedisQueue) Push(task *models.Task) error { |  | ||||||
| 	bs, err := json.Marshal(task) |  | ||||||
| 	if err != nil { |  | ||||||
| 		return err |  | ||||||
| 	} |  | ||||||
| 	return r.client.RPush(r.queueName, bs).Err() |  | ||||||
| } |  | ||||||
|  |  | ||||||
| // Stop stop the queue |  | ||||||
| func (r *RedisQueue) Stop() { |  | ||||||
| 	r.closeChan <- true |  | ||||||
| } |  | ||||||
| @@ -8,14 +8,15 @@ import ( | |||||||
| 	"fmt" | 	"fmt" | ||||||
|  |  | ||||||
| 	"code.gitea.io/gitea/models" | 	"code.gitea.io/gitea/models" | ||||||
|  | 	"code.gitea.io/gitea/modules/graceful" | ||||||
| 	"code.gitea.io/gitea/modules/log" | 	"code.gitea.io/gitea/modules/log" | ||||||
| 	"code.gitea.io/gitea/modules/migrations/base" | 	"code.gitea.io/gitea/modules/migrations/base" | ||||||
| 	"code.gitea.io/gitea/modules/setting" | 	"code.gitea.io/gitea/modules/queue" | ||||||
| 	"code.gitea.io/gitea/modules/structs" | 	"code.gitea.io/gitea/modules/structs" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| // taskQueue is a global queue of tasks | // taskQueue is a global queue of tasks | ||||||
| var taskQueue Queue | var taskQueue queue.Queue | ||||||
|  |  | ||||||
| // Run a task | // Run a task | ||||||
| func Run(t *models.Task) error { | func Run(t *models.Task) error { | ||||||
| @@ -23,38 +24,32 @@ func Run(t *models.Task) error { | |||||||
| 	case structs.TaskTypeMigrateRepo: | 	case structs.TaskTypeMigrateRepo: | ||||||
| 		return runMigrateTask(t) | 		return runMigrateTask(t) | ||||||
| 	default: | 	default: | ||||||
| 		return fmt.Errorf("Unknow task type: %d", t.Type) | 		return fmt.Errorf("Unknown task type: %d", t.Type) | ||||||
| 	} | 	} | ||||||
| } | } | ||||||
|  |  | ||||||
| // Init will start the service to get all unfinished tasks and run them | // Init will start the service to get all unfinished tasks and run them | ||||||
| func Init() error { | func Init() error { | ||||||
| 	switch setting.Task.QueueType { | 	taskQueue = queue.CreateQueue("task", handle, &models.Task{}) | ||||||
| 	case setting.ChannelQueueType: |  | ||||||
| 		taskQueue = NewChannelQueue(setting.Task.QueueLength) | 	if taskQueue == nil { | ||||||
| 	case setting.RedisQueueType: | 		return fmt.Errorf("Unable to create Task Queue") | ||||||
| 		var err error |  | ||||||
| 		addrs, pass, idx, err := parseConnStr(setting.Task.QueueConnStr) |  | ||||||
| 		if err != nil { |  | ||||||
| 			return err |  | ||||||
| 		} |  | ||||||
| 		taskQueue, err = NewRedisQueue(addrs, pass, idx) |  | ||||||
| 		if err != nil { |  | ||||||
| 			return err |  | ||||||
| 		} |  | ||||||
| 	default: |  | ||||||
| 		return fmt.Errorf("Unsupported task queue type: %v", setting.Task.QueueType) |  | ||||||
| 	} | 	} | ||||||
|  |  | ||||||
| 	go func() { | 	go graceful.GetManager().RunWithShutdownFns(taskQueue.Run) | ||||||
| 		if err := taskQueue.Run(); err != nil { |  | ||||||
| 			log.Error("taskQueue.Run end failed: %v", err) |  | ||||||
| 		} |  | ||||||
| 	}() |  | ||||||
|  |  | ||||||
| 	return nil | 	return nil | ||||||
| } | } | ||||||
|  |  | ||||||
|  | func handle(data ...queue.Data) { | ||||||
|  | 	for _, datum := range data { | ||||||
|  | 		task := datum.(*models.Task) | ||||||
|  | 		if err := Run(task); err != nil { | ||||||
|  | 			log.Error("Run task failed: %v", err) | ||||||
|  | 		} | ||||||
|  | 	} | ||||||
|  | } | ||||||
|  |  | ||||||
| // MigrateRepository add migration repository to task | // MigrateRepository add migration repository to task | ||||||
| func MigrateRepository(doer, u *models.User, opts base.MigrateOptions) error { | func MigrateRepository(doer, u *models.User, opts base.MigrateOptions) error { | ||||||
| 	task, err := models.CreateMigrateTask(doer, u, opts) | 	task, err := models.CreateMigrateTask(doer, u, opts) | ||||||
|   | |||||||
| @@ -2026,6 +2026,54 @@ monitor.execute_time = Execution Time | |||||||
| monitor.process.cancel = Cancel process | monitor.process.cancel = Cancel process | ||||||
| monitor.process.cancel_desc =  Cancelling a process may cause data loss | monitor.process.cancel_desc =  Cancelling a process may cause data loss | ||||||
| monitor.process.cancel_notices =  Cancel: <strong>%s</strong>? | monitor.process.cancel_notices =  Cancel: <strong>%s</strong>? | ||||||
|  | monitor.queues = Queues | ||||||
|  | monitor.queue = Queue: %s | ||||||
|  | monitor.queue.name = Name | ||||||
|  | monitor.queue.type = Type | ||||||
|  | monitor.queue.exemplar = Exemplar Type | ||||||
|  | monitor.queue.numberworkers = Number of Workers | ||||||
|  | monitor.queue.maxnumberworkers = Max Number of Workers | ||||||
|  | monitor.queue.review = Review Config | ||||||
|  | monitor.queue.review_add = Review/Add Workers | ||||||
|  | monitor.queue.configuration = Initial Configuration | ||||||
|  | monitor.queue.nopool.title = No Worker Pool | ||||||
|  | monitor.queue.nopool.desc = This queue wraps other queues and does not itself have a worker pool. | ||||||
|  | monitor.queue.wrapped.desc = A wrapped queue wraps a slow starting queue, buffering queued requests in a channel. It does not have a worker pool itself. | ||||||
|  | monitor.queue.persistable-channel.desc = A persistable-channel wraps two queues, a channel queue that has its own worker pool and a level queue for persisted requests from previous shutdowns. It does not have a worker pool itself. | ||||||
|  | monitor.queue.pool.timeout = Timeout | ||||||
|  | monitor.queue.pool.addworkers.title = Add Workers | ||||||
|  | monitor.queue.pool.addworkers.submit = Add Workers | ||||||
|  | monitor.queue.pool.addworkers.desc = Add Workers to this pool with or without a timeout. If you set a timeout these workers will be removed from the pool after the timeout has lapsed. | ||||||
|  | monitor.queue.pool.addworkers.numberworkers.placeholder = Number of Workers | ||||||
|  | monitor.queue.pool.addworkers.timeout.placeholder = Set to 0 for no timeout | ||||||
|  | monitor.queue.pool.addworkers.mustnumbergreaterzero = Number of Workers to add must be greater than zero | ||||||
|  | monitor.queue.pool.addworkers.musttimeoutduration = Timeout must be a golang duration eg. 5m or be 0 | ||||||
|  |  | ||||||
|  | monitor.queue.settings.title = Pool Settings | ||||||
|  | monitor.queue.settings.desc = Pools dynamically grow with a boost in response to their worker queue blocking. These changes will not affect current worker groups. | ||||||
|  | monitor.queue.settings.timeout = Boost Timeout | ||||||
|  | monitor.queue.settings.timeout.placeholder = Currently %[1]v | ||||||
|  | monitor.queue.settings.timeout.error = Timeout must be a golang duration eg. 5m or be 0 | ||||||
|  | monitor.queue.settings.numberworkers = Boost Number of Workers | ||||||
|  | monitor.queue.settings.numberworkers.placeholder = Currently %[1]d | ||||||
|  | monitor.queue.settings.numberworkers.error = Number of Workers to add must be greater than or equal to zero | ||||||
|  | monitor.queue.settings.maxnumberworkers = Max Number of workers | ||||||
|  | monitor.queue.settings.maxnumberworkers.placeholder = Currently %[1]d | ||||||
|  | monitor.queue.settings.maxnumberworkers.error = Max number of workers must be a number | ||||||
|  | monitor.queue.settings.submit = Update Settings | ||||||
|  | monitor.queue.settings.changed = Settings Updated | ||||||
|  | monitor.queue.settings.blocktimeout = Current Block Timeout | ||||||
|  | monitor.queue.settings.blocktimeout.value = %[1]v | ||||||
|  |  | ||||||
|  | monitor.queue.pool.none = This queue does not have a Pool | ||||||
|  | monitor.queue.pool.added = Worker Group Added | ||||||
|  | monitor.queue.pool.max_changed = Maximum number of workers changed | ||||||
|  | monitor.queue.pool.workers.title = Active Worker Groups | ||||||
|  | monitor.queue.pool.workers.none = No worker groups. | ||||||
|  | monitor.queue.pool.cancel = Shutdown Worker Group | ||||||
|  | monitor.queue.pool.cancelling = Worker Group shutting down | ||||||
|  | monitor.queue.pool.cancel_notices = Shutdown this group of %s workers? | ||||||
|  | monitor.queue.pool.cancel_desc = Leaving a queue without any worker groups may cause requests to block indefinitely. | ||||||
|  |  | ||||||
| notices.system_notice_list = System Notices | notices.system_notice_list = System Notices | ||||||
| notices.view_detail_header = View Notice Details | notices.view_detail_header = View Notice Details | ||||||
|   | |||||||
| @@ -11,6 +11,7 @@ import ( | |||||||
| 	"net/url" | 	"net/url" | ||||||
| 	"os" | 	"os" | ||||||
| 	"runtime" | 	"runtime" | ||||||
|  | 	"strconv" | ||||||
| 	"strings" | 	"strings" | ||||||
| 	"time" | 	"time" | ||||||
|  |  | ||||||
| @@ -22,6 +23,7 @@ import ( | |||||||
| 	"code.gitea.io/gitea/modules/graceful" | 	"code.gitea.io/gitea/modules/graceful" | ||||||
| 	"code.gitea.io/gitea/modules/log" | 	"code.gitea.io/gitea/modules/log" | ||||||
| 	"code.gitea.io/gitea/modules/process" | 	"code.gitea.io/gitea/modules/process" | ||||||
|  | 	"code.gitea.io/gitea/modules/queue" | ||||||
| 	"code.gitea.io/gitea/modules/setting" | 	"code.gitea.io/gitea/modules/setting" | ||||||
| 	"code.gitea.io/gitea/modules/timeutil" | 	"code.gitea.io/gitea/modules/timeutil" | ||||||
| 	"code.gitea.io/gitea/services/mailer" | 	"code.gitea.io/gitea/services/mailer" | ||||||
| @@ -35,6 +37,7 @@ const ( | |||||||
| 	tplDashboard base.TplName = "admin/dashboard" | 	tplDashboard base.TplName = "admin/dashboard" | ||||||
| 	tplConfig    base.TplName = "admin/config" | 	tplConfig    base.TplName = "admin/config" | ||||||
| 	tplMonitor   base.TplName = "admin/monitor" | 	tplMonitor   base.TplName = "admin/monitor" | ||||||
|  | 	tplQueue     base.TplName = "admin/queue" | ||||||
| ) | ) | ||||||
|  |  | ||||||
| var ( | var ( | ||||||
| @@ -355,6 +358,7 @@ func Monitor(ctx *context.Context) { | |||||||
| 	ctx.Data["PageIsAdminMonitor"] = true | 	ctx.Data["PageIsAdminMonitor"] = true | ||||||
| 	ctx.Data["Processes"] = process.GetManager().Processes() | 	ctx.Data["Processes"] = process.GetManager().Processes() | ||||||
| 	ctx.Data["Entries"] = cron.ListTasks() | 	ctx.Data["Entries"] = cron.ListTasks() | ||||||
|  | 	ctx.Data["Queues"] = queue.GetManager().ManagedQueues() | ||||||
| 	ctx.HTML(200, tplMonitor) | 	ctx.HTML(200, tplMonitor) | ||||||
| } | } | ||||||
|  |  | ||||||
| @@ -366,3 +370,126 @@ func MonitorCancel(ctx *context.Context) { | |||||||
| 		"redirect": ctx.Repo.RepoLink + "/admin/monitor", | 		"redirect": ctx.Repo.RepoLink + "/admin/monitor", | ||||||
| 	}) | 	}) | ||||||
| } | } | ||||||
|  |  | ||||||
|  | // Queue shows details for a specific queue | ||||||
|  | func Queue(ctx *context.Context) { | ||||||
|  | 	qid := ctx.ParamsInt64("qid") | ||||||
|  | 	mq := queue.GetManager().GetManagedQueue(qid) | ||||||
|  | 	if mq == nil { | ||||||
|  | 		ctx.Status(404) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	ctx.Data["Title"] = ctx.Tr("admin.monitor.queue", mq.Name) | ||||||
|  | 	ctx.Data["PageIsAdmin"] = true | ||||||
|  | 	ctx.Data["PageIsAdminMonitor"] = true | ||||||
|  | 	ctx.Data["Queue"] = mq | ||||||
|  | 	ctx.HTML(200, tplQueue) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // WorkerCancel cancels a worker group | ||||||
|  | func WorkerCancel(ctx *context.Context) { | ||||||
|  | 	qid := ctx.ParamsInt64("qid") | ||||||
|  | 	mq := queue.GetManager().GetManagedQueue(qid) | ||||||
|  | 	if mq == nil { | ||||||
|  | 		ctx.Status(404) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	pid := ctx.ParamsInt64("pid") | ||||||
|  | 	mq.CancelWorkers(pid) | ||||||
|  | 	ctx.Flash.Info(ctx.Tr("admin.monitor.queue.pool.cancelling")) | ||||||
|  | 	ctx.JSON(200, map[string]interface{}{ | ||||||
|  | 		"redirect": setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid), | ||||||
|  | 	}) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // AddWorkers adds workers to a worker group | ||||||
|  | func AddWorkers(ctx *context.Context) { | ||||||
|  | 	qid := ctx.ParamsInt64("qid") | ||||||
|  | 	mq := queue.GetManager().GetManagedQueue(qid) | ||||||
|  | 	if mq == nil { | ||||||
|  | 		ctx.Status(404) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	number := ctx.QueryInt("number") | ||||||
|  | 	if number < 1 { | ||||||
|  | 		ctx.Flash.Error(ctx.Tr("admin.monitor.queue.pool.addworkers.mustnumbergreaterzero")) | ||||||
|  | 		ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	timeout, err := time.ParseDuration(ctx.Query("timeout")) | ||||||
|  | 	if err != nil { | ||||||
|  | 		ctx.Flash.Error(ctx.Tr("admin.monitor.queue.pool.addworkers.musttimeoutduration")) | ||||||
|  | 		ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	if mq.Pool == nil { | ||||||
|  | 		ctx.Flash.Error(ctx.Tr("admin.monitor.queue.pool.none")) | ||||||
|  | 		ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	mq.AddWorkers(number, timeout) | ||||||
|  | 	ctx.Flash.Success(ctx.Tr("admin.monitor.queue.pool.added")) | ||||||
|  | 	ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | } | ||||||
|  |  | ||||||
|  | // SetQueueSettings sets the maximum number of workers and other settings for this queue | ||||||
|  | func SetQueueSettings(ctx *context.Context) { | ||||||
|  | 	qid := ctx.ParamsInt64("qid") | ||||||
|  | 	mq := queue.GetManager().GetManagedQueue(qid) | ||||||
|  | 	if mq == nil { | ||||||
|  | 		ctx.Status(404) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  | 	if mq.Pool == nil { | ||||||
|  | 		ctx.Flash.Error(ctx.Tr("admin.monitor.queue.pool.none")) | ||||||
|  | 		ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 		return | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	maxNumberStr := ctx.Query("max-number") | ||||||
|  | 	numberStr := ctx.Query("number") | ||||||
|  | 	timeoutStr := ctx.Query("timeout") | ||||||
|  |  | ||||||
|  | 	var err error | ||||||
|  | 	var maxNumber, number int | ||||||
|  | 	var timeout time.Duration | ||||||
|  | 	if len(maxNumberStr) > 0 { | ||||||
|  | 		maxNumber, err = strconv.Atoi(maxNumberStr) | ||||||
|  | 		if err != nil { | ||||||
|  | 			ctx.Flash.Error(ctx.Tr("admin.monitor.queue.settings.maxnumberworkers.error")) | ||||||
|  | 			ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 		if maxNumber < -1 { | ||||||
|  | 			maxNumber = -1 | ||||||
|  | 		} | ||||||
|  | 	} else { | ||||||
|  | 		maxNumber = mq.MaxNumberOfWorkers() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if len(numberStr) > 0 { | ||||||
|  | 		number, err = strconv.Atoi(numberStr) | ||||||
|  | 		if err != nil || number < 0 { | ||||||
|  | 			ctx.Flash.Error(ctx.Tr("admin.monitor.queue.settings.numberworkers.error")) | ||||||
|  | 			ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 	} else { | ||||||
|  | 		number = mq.BoostWorkers() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	if len(timeoutStr) > 0 { | ||||||
|  | 		timeout, err = time.ParseDuration(timeoutStr) | ||||||
|  | 		if err != nil { | ||||||
|  | 			ctx.Flash.Error(ctx.Tr("admin.monitor.queue.settings.timeout.error")) | ||||||
|  | 			ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | 			return | ||||||
|  | 		} | ||||||
|  | 	} else { | ||||||
|  | 		timeout = mq.Pool.BoostTimeout() | ||||||
|  | 	} | ||||||
|  |  | ||||||
|  | 	mq.SetSettings(maxNumber, number, timeout) | ||||||
|  | 	ctx.Flash.Success(ctx.Tr("admin.monitor.queue.settings.changed")) | ||||||
|  | 	ctx.Redirect(setting.AppSubURL + fmt.Sprintf("/admin/monitor/queue/%d", qid)) | ||||||
|  | } | ||||||
|   | |||||||
| @@ -410,8 +410,16 @@ func RegisterRoutes(m *macaron.Macaron) { | |||||||
| 		m.Get("", adminReq, admin.Dashboard) | 		m.Get("", adminReq, admin.Dashboard) | ||||||
| 		m.Get("/config", admin.Config) | 		m.Get("/config", admin.Config) | ||||||
| 		m.Post("/config/test_mail", admin.SendTestMail) | 		m.Post("/config/test_mail", admin.SendTestMail) | ||||||
| 		m.Get("/monitor", admin.Monitor) | 		m.Group("/monitor", func() { | ||||||
| 		m.Post("/monitor/cancel/:pid", admin.MonitorCancel) | 			m.Get("", admin.Monitor) | ||||||
|  | 			m.Post("/cancel/:pid", admin.MonitorCancel) | ||||||
|  | 			m.Group("/queue/:qid", func() { | ||||||
|  | 				m.Get("", admin.Queue) | ||||||
|  | 				m.Post("/set", admin.SetQueueSettings) | ||||||
|  | 				m.Post("/add", admin.AddWorkers) | ||||||
|  | 				m.Post("/cancel/:pid", admin.WorkerCancel) | ||||||
|  | 			}) | ||||||
|  | 		}) | ||||||
|  |  | ||||||
| 		m.Group("/users", func() { | 		m.Group("/users", func() { | ||||||
| 			m.Get("", admin.Users) | 			m.Get("", admin.Users) | ||||||
|   | |||||||
| @@ -31,6 +31,34 @@ | |||||||
| 			</table> | 			</table> | ||||||
| 		</div> | 		</div> | ||||||
|  |  | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queues"}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached table segment"> | ||||||
|  | 			<table class="ui very basic striped table"> | ||||||
|  | 				<thead> | ||||||
|  | 					<tr> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.name"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.type"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.exemplar"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.numberworkers"}}</th> | ||||||
|  | 						<th></th> | ||||||
|  | 					</tr> | ||||||
|  | 				</thead> | ||||||
|  | 				<tbody> | ||||||
|  | 					{{range .Queues}} | ||||||
|  | 						<tr> | ||||||
|  | 							<td>{{.Name}}</td> | ||||||
|  | 							<td>{{.Type}}</td> | ||||||
|  | 							<td>{{.ExemplarType}}</td> | ||||||
|  | 							<td>{{$sum := .NumberOfWorkers}}{{if lt $sum 0}}-{{else}}{{$sum}}{{end}}</td> | ||||||
|  | 							<td><a href="{{$.Link}}/queue/{{.QID}}" class="button">{{if lt $sum 0}}{{$.i18n.Tr "admin.monitor.queue.review"}}{{else}}{{$.i18n.Tr "admin.monitor.queue.review_add"}}{{end}}</a> | ||||||
|  | 						</tr> | ||||||
|  | 					{{end}} | ||||||
|  | 				</tbody> | ||||||
|  | 			</table> | ||||||
|  | 		</div> | ||||||
|  |  | ||||||
| 		<h4 class="ui top attached header"> | 		<h4 class="ui top attached header"> | ||||||
| 			{{.i18n.Tr "admin.monitor.process"}} | 			{{.i18n.Tr "admin.monitor.process"}} | ||||||
| 		</h4> | 		</h4> | ||||||
|   | |||||||
							
								
								
									
										147
									
								
								templates/admin/queue.tmpl
									
									
									
									
									
										Normal file
									
								
							
							
						
						
									
										147
									
								
								templates/admin/queue.tmpl
									
									
									
									
									
										Normal file
									
								
							| @@ -0,0 +1,147 @@ | |||||||
|  | {{template "base/head" .}} | ||||||
|  | <div class="admin monitor"> | ||||||
|  | 	{{template "admin/navbar" .}} | ||||||
|  | 	<div class="ui container"> | ||||||
|  | 		{{template "base/alert" .}} | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queue" .Queue.Name}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached table segment"> | ||||||
|  | 			<table class="ui very basic striped table"> | ||||||
|  | 				<thead> | ||||||
|  | 					<tr> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.name"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.type"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.exemplar"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.numberworkers"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.maxnumberworkers"}}</th> | ||||||
|  | 					</tr> | ||||||
|  | 				</thead> | ||||||
|  | 				<tbody> | ||||||
|  | 					<tr> | ||||||
|  | 						<td>{{.Queue.Name}}</td> | ||||||
|  | 						<td>{{.Queue.Type}}</td> | ||||||
|  | 						<td>{{.Queue.ExemplarType}}</td> | ||||||
|  | 						<td>{{$sum := .Queue.NumberOfWorkers}}{{if lt $sum 0}}-{{else}}{{$sum}}{{end}}</td> | ||||||
|  | 						<td>{{if lt $sum 0}}-{{else}}{{.Queue.MaxNumberOfWorkers}}{{end}}</td> | ||||||
|  | 					</tr> | ||||||
|  | 				</tbody> | ||||||
|  | 			</table> | ||||||
|  | 		</div> | ||||||
|  | 		{{if lt $sum 0 }} | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queue.nopool.title"}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached segment"> | ||||||
|  | 			{{if eq .Queue.Type "wrapped" }} | ||||||
|  | 			<p>{{.i18n.Tr "admin.monitor.queue.wrapped.desc"}}</p> | ||||||
|  | 			{{else if eq .Queue.Type "persistable-channel"}} | ||||||
|  | 			<p>{{.i18n.Tr "admin.monitor.queue.persistable-channel.desc"}}</p> | ||||||
|  | 			{{else}} | ||||||
|  | 			<p>{{.i18n.Tr "admin.monitor.queue.nopool.desc"}}</p> | ||||||
|  | 			{{end}} | ||||||
|  | 		</div> | ||||||
|  | 		{{else}} | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queue.settings.title"}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached segment"> | ||||||
|  | 			<p>{{.i18n.Tr "admin.monitor.queue.settings.desc"}}</p> | ||||||
|  | 			<form method="POST" action="{{.Link}}/set"> | ||||||
|  | 				{{$.CsrfTokenHtml}} | ||||||
|  | 				<div class="ui form"> | ||||||
|  | 					<div class="inline field"> | ||||||
|  | 						<label for="max-number">{{.i18n.Tr "admin.monitor.queue.settings.maxnumberworkers"}}</label> | ||||||
|  | 						<input name="max-number" type="text" placeholder="{{.i18n.Tr "admin.monitor.queue.settings.maxnumberworkers.placeholder" .Queue.MaxNumberOfWorkers}}"> | ||||||
|  | 					</div> | ||||||
|  | 					<div class="inline field"> | ||||||
|  | 						<label for="timeout">{{.i18n.Tr "admin.monitor.queue.settings.timeout"}}</label> | ||||||
|  | 						<input name="timeout" type="text" placeholder="{{.i18n.Tr "admin.monitor.queue.settings.timeout.placeholder" .Queue.BoostTimeout }}"> | ||||||
|  | 					</div> | ||||||
|  | 					<div class="inline field"> | ||||||
|  | 						<label for="number">{{.i18n.Tr "admin.monitor.queue.settings.numberworkers"}}</label> | ||||||
|  | 						<input name="number" type="text" placeholder="{{.i18n.Tr "admin.monitor.queue.settings.numberworkers.placeholder" .Queue.BoostWorkers}}"> | ||||||
|  | 					</div> | ||||||
|  | 					<div class="inline field"> | ||||||
|  | 						<label>{{.i18n.Tr "admin.monitor.queue.settings.blocktimeout"}}</label> | ||||||
|  | 						<span>{{.i18n.Tr "admin.monitor.queue.settings.blocktimeout.value" .Queue.BlockTimeout}}</span> | ||||||
|  | 					</div> | ||||||
|  | 					<button class="ui submit button">{{.i18n.Tr "admin.monitor.queue.settings.submit"}}</button> | ||||||
|  | 				</div> | ||||||
|  | 			</form> | ||||||
|  | 		</div> | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queue.pool.addworkers.title"}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached segment"> | ||||||
|  | 			<p>{{.i18n.Tr "admin.monitor.queue.pool.addworkers.desc"}}</p> | ||||||
|  | 			<form method="POST" action="{{.Link}}/add"> | ||||||
|  | 				{{$.CsrfTokenHtml}} | ||||||
|  | 				<div class="ui form"> | ||||||
|  | 					<div class="fields"> | ||||||
|  | 						<div class="field"> | ||||||
|  | 							<label>{{.i18n.Tr "admin.monitor.queue.numberworkers"}}</label> | ||||||
|  | 							<input name="number" type="text" placeholder="{{.i18n.Tr "admin.monitor.queue.pool.addworkers.numberworkers.placeholder"}}"> | ||||||
|  | 						</div> | ||||||
|  | 						<div class="field"> | ||||||
|  | 							<label>{{.i18n.Tr "admin.monitor.queue.pool.timeout"}}</label> | ||||||
|  | 							<input name="timeout" type="text" placeholder="{{.i18n.Tr "admin.monitor.queue.pool.addworkers.timeout.placeholder"}}"> | ||||||
|  | 						</div> | ||||||
|  | 					</div> | ||||||
|  | 					<button class="ui submit button">{{.i18n.Tr "admin.monitor.queue.pool.addworkers.submit"}}</button> | ||||||
|  | 				</div> | ||||||
|  | 			</form> | ||||||
|  | 		</div> | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queue.pool.workers.title"}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached table segment"> | ||||||
|  | 			<table class="ui very basic striped table"> | ||||||
|  | 				<thead> | ||||||
|  | 					<tr> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.numberworkers"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.start"}}</th> | ||||||
|  | 						<th>{{.i18n.Tr "admin.monitor.queue.pool.timeout"}}</th> | ||||||
|  | 						<th></th> | ||||||
|  | 					</tr> | ||||||
|  | 				</thead> | ||||||
|  | 				<tbody> | ||||||
|  | 					{{range .Queue.Workers}} | ||||||
|  | 					<tr> | ||||||
|  | 						<td>{{.Workers}}</td> | ||||||
|  | 						<td>{{DateFmtLong .Start}}</td> | ||||||
|  | 						<td>{{if .HasTimeout}}{{DateFmtLong .Timeout}}{{else}}-{{end}}</td> | ||||||
|  | 						<td> | ||||||
|  | 							<a class="delete-button" href="" data-url="{{$.Link}}/cancel/{{.PID}}" data-id="{{.PID}}" data-name="{{.Workers}}"><i class="close icon text red" title="{{$.i18n.Tr "remove"}}"></i></a> | ||||||
|  | 						</td> | ||||||
|  | 					</tr> | ||||||
|  | 					{{else}} | ||||||
|  | 						<tr> | ||||||
|  | 							<td colspan="4">{{.i18n.Tr "admin.monitor.queue.pool.workers.none" }} | ||||||
|  | 						</tr> | ||||||
|  | 					{{end}} | ||||||
|  | 				</tbody> | ||||||
|  | 			</table> | ||||||
|  | 		</div> | ||||||
|  | 		{{end}} | ||||||
|  | 		<h4 class="ui top attached header"> | ||||||
|  | 			{{.i18n.Tr "admin.monitor.queue.configuration"}} | ||||||
|  | 		</h4> | ||||||
|  | 		<div class="ui attached segment"> | ||||||
|  | 			<pre>{{.Queue.Configuration | JsonPrettyPrint}} | ||||||
|  | 		</div> | ||||||
|  | 	</div> | ||||||
|  | </div> | ||||||
|  | <div class="ui small basic delete modal"> | ||||||
|  | 	<div class="ui icon header"> | ||||||
|  | 		<i class="close icon"></i> | ||||||
|  | 		{{.i18n.Tr "admin.monitor.queue.pool.cancel"}} | ||||||
|  | 	</div> | ||||||
|  | 	<div class="content"> | ||||||
|  | 		<p>{{$.i18n.Tr "admin.monitor.queue.pool.cancel_notices" `<span class="name"></span>` | Safe}}</p> | ||||||
|  | 		<p>{{$.i18n.Tr "admin.monitor.queue.pool.cancel_desc"}}</p> | ||||||
|  | 	</div> | ||||||
|  | 	{{template "base/delete_modal_actions" .}} | ||||||
|  | </div> | ||||||
|  |  | ||||||
|  | {{template "base/footer" .}} | ||||||
		Reference in New Issue
	
	Block a user