Welcome to mirror list, hosted at ThFree Co, Russian Federation.

gitlab.com/gitlab-org/gitlab-pages.git - Unnamed repository; edit this file 'description' to name the repository.
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorGrzegorz Bizon <grzesiek.bizon@gmail.com>2019-11-12 16:17:29 +0300
committerGrzegorz Bizon <grzesiek.bizon@gmail.com>2019-11-12 16:17:29 +0300
commitb60ee3425b4b6d54154de1629075b52b8599d869 (patch)
treea96eae38f4427a45bdbe00af0dbc39656871e001
parent4e0be9393f33d27ea381cae3e6a6aeda88032153 (diff)
Refactor gitlab source cache to make it more thread safe
-rw-r--r--internal/source/gitlab/cache.go151
-rw-r--r--internal/source/gitlab/cache/cache.go38
-rw-r--r--internal/source/gitlab/cache/cache_test.go164
-rw-r--r--internal/source/gitlab/cache/entry.go120
-rw-r--r--internal/source/gitlab/cache/lookup.go19
-rw-r--r--internal/source/gitlab/cache/memstore.go59
-rw-r--r--internal/source/gitlab/cache/resolver.go8
-rw-r--r--internal/source/gitlab/cache/response.go13
-rw-r--r--internal/source/gitlab/cache/retriever.go49
-rw-r--r--internal/source/gitlab/cache/store.go9
-rw-r--r--internal/source/gitlab/cache_test.go119
-rw-r--r--internal/source/gitlab/client.go4
-rw-r--r--internal/source/gitlab/gitlab.go3
-rw-r--r--internal/source/gitlab/lookup.go16
14 files changed, 485 insertions, 287 deletions
diff --git a/internal/source/gitlab/cache.go b/internal/source/gitlab/cache.go
deleted file mode 100644
index ad0b13dc..00000000
--- a/internal/source/gitlab/cache.go
+++ /dev/null
@@ -1,151 +0,0 @@
-package gitlab
-
-import (
- "context"
- "fmt"
- "sync"
- "time"
-
- cache "github.com/patrickmn/go-cache"
-)
-
-// Cache is a short and long caching mechanism for GitLab source
-type Cache struct {
- shortCache *cache.Cache
- longCache *cache.Cache
- lockCache *sync.Map
-}
-
-type lock struct {
- time time.Time
- once *sync.Once
- cond *sync.Cond
-}
-
-type retrieveFunc func(ctx context.Context) (Lookup, error)
-
-var (
- maxRetrievalInterval = time.Second
- maxRetrievalTimeout = 4 * time.Second
-)
-
-// NewCache creates a new instance of Cache and sets default expiration.
-// Short cache expiration - 5 seconds
-// Long cache expiration - 10 minutes
-// Cache cleanup interval - 1 minute
-func NewCache() *Cache {
- return &Cache{
- shortCache: cache.New(5*time.Second, time.Minute),
- longCache: cache.New(10*time.Minute, time.Minute),
- lockCache: &sync.Map{},
- }
-}
-
-// GetLookup is going to return a Lookup based on a domain name using following
-// algorithm:
-// - if a domain lookup is present in the short cache it will return just it
-// - if it is not present in a short cache it will check the long cache
-// - if it is present in a long cache it will return the long cache version and
-// run an update in a separate thread that will fetch the lookup from the
-// GitLab source and replace the short and long cache entries
-// - if a domain lookup is not present in the long cache we will fetch the
-// lookup from the domain source and client will need to wait
-// TODO add error handling to Lookup
-func (c *Cache) GetLookup(domain string, retriever retrieveFunc) *Lookup {
- // return lookup if it exists in the short cache
- if lookup, exists := c.shortCache.Get(domain); exists {
- return lookup.(*Lookup)
- }
-
- // return lookup it if exists in the long cache, schedule retrieval
- if lookup, exists := c.longCache.Get(domain); exists {
- c.withLock(domain, func(lock *lock) {
- lock.once.Do(func() { go c.retrieve(domain, retriever) })
- })
-
- return lookup.(*Lookup)
- }
-
- // perform retrieval once and wait for the response
- c.withLock(domain, func(lock *lock) {
- lock.once.Do(func() { go c.retrieve(domain, retriever) })
- lock.cond.Wait()
- })
-
- fmt.Println("GetLookup")
- return c.GetLookup(domain, retriever)
-}
-
-func (c *Cache) retrieve(domain string, retriever retrieveFunc) {
- var lookup Lookup
- response := make(chan Lookup)
- defer close(response)
-
- ctx, cancel := context.WithTimeout(context.Background(), maxRetrievalTimeout)
- defer cancel()
-
- go c.retrievalLoop(ctx, retriever, response)
-
- select {
- case <-ctx.Done():
- lookup = Lookup{} // TODO store error
- case lookup = <-response:
- fmt.Println("response received") // TODO Log response
- }
-
- c.withLock(domain, func(lock *lock) {
- c.store(domain, lookup)
- c.lockCache.Delete(domain)
- lock.cond.Broadcast() // broadcast lookup message to all listeners
- })
-}
-
-func (c *Cache) retrievalLoop(ctx context.Context, retriever retrieveFunc, response chan<- Lookup) {
- for {
- if ctx.Err() != nil {
- return
- }
-
- lookup, err := retriever(ctx)
-
- if err != nil {
- time.Sleep(maxRetrievalInterval)
- } else {
- response <- lookup
- return
- }
- }
-}
-
-func (c *Cache) withLock(domain string, block func(*lock)) {
- newLock := &lock{
- time: time.Now(),
- once: &sync.Once{},
- cond: sync.NewCond(&sync.Mutex{}),
- }
-
- // go-cache does not have atomic load and store
- storedLock, _ := c.lockCache.LoadOrStore(domain, newLock)
- cacheLock := storedLock.(*lock)
-
- if cacheLock.isExpired() { // custom lock expiration
- c.lockCache.Delete(domain) // remove expired lock
- c.withLock(domain, block) // retry aquiring lock
- return
- }
-
- cacheLock.cond.L.Lock()
- block(cacheLock)
- cacheLock.cond.L.Unlock()
-}
-
-func (c *Cache) store(domain string, lookup Lookup) *Lookup {
- c.shortCache.SetDefault(domain, &lookup)
- c.longCache.SetDefault(domain, &lookup)
-
- return &lookup
-}
-
-func (l *lock) isExpired() bool {
- return l.time.Add(10 * time.Second).Before(time.Now())
-}
diff --git a/internal/source/gitlab/cache/cache.go b/internal/source/gitlab/cache/cache.go
new file mode 100644
index 00000000..4ffc6022
--- /dev/null
+++ b/internal/source/gitlab/cache/cache.go
@@ -0,0 +1,38 @@
+package cache
+
+import (
+ "context"
+)
+
+// Cache is a short and long caching mechanism for GitLab source
+type Cache struct {
+ client Resolver
+ store Store
+}
+
+// NewCache creates a new instance of Cache.
+func NewCache(client Resolver) *Cache {
+ return &Cache{
+ client: client,
+ store: newMemStore(),
+ }
+}
+
+// Resolve is going to return a Lookup based on a domain name
+func (c *Cache) Resolve(ctx context.Context, domain string) (*Lookup, int, error) {
+ entry := c.store.LoadOrCreate(ctx, domain)
+
+ if entry.IsUpToDate() {
+ return entry.Lookup()
+ }
+
+ if entry.NeedsRefresh() {
+ entry.Refresh(ctx, c.client, c.store)
+
+ return entry.Lookup()
+ }
+
+ <-entry.Retrieve(c.client)
+
+ return c.Resolve(ctx, domain)
+}
diff --git a/internal/source/gitlab/cache/cache_test.go b/internal/source/gitlab/cache/cache_test.go
new file mode 100644
index 00000000..f5dd0dfa
--- /dev/null
+++ b/internal/source/gitlab/cache/cache_test.go
@@ -0,0 +1,164 @@
+package cache
+
+import (
+ "context"
+ "sync"
+ "sync/atomic"
+ "testing"
+ "time"
+
+ "github.com/stretchr/testify/assert"
+)
+
+type client struct {
+ resolutions uint64
+ domain chan string
+ failure error
+ status int
+}
+
+func (s *client) Resolve(ctx context.Context, domain string) (*Lookup, int, error) {
+ atomic.AddUint64(&s.resolutions, 1)
+
+ if s.failure != nil {
+ return &Lookup{}, s.status, s.failure
+ }
+
+ return &Lookup{Domain: <-s.domain}, 200, nil
+}
+
+func withTestCache(config resolverConfig, block func(*Cache, *client)) {
+ var resolver *client
+
+ if config.buffered {
+ resolver = &client{domain: make(chan string, 1)}
+ } else {
+ resolver = &client{domain: make(chan string)}
+ }
+
+ cache := NewCache(resolver)
+
+ block(cache, resolver)
+}
+
+func (cache *Cache) withTestEntry(config entryConfig, block func()) {
+ domain := "my.gitlab.com"
+
+ if len(config.domain) > 0 {
+ domain = config.domain
+ }
+
+ entry := cache.store.ReplaceOrCreate(context.Background(), domain)
+
+ if config.retrieved {
+ newResponse := make(chan Response, 1)
+ newResponse <- Response{lookup: &Lookup{Domain: domain}, status: 200}
+ entry.setResponse(newResponse)
+ }
+
+ if config.expired {
+ entry.created = time.Now().Add(-time.Hour)
+ }
+
+ block()
+}
+
+type resolverConfig struct {
+ buffered bool
+}
+
+type entryConfig struct {
+ domain string
+ expired bool
+ retrieved bool
+}
+
+func TestGetLookup(t *testing.T) {
+ t.Run("when item is not cached", func(t *testing.T) {
+ withTestCache(resolverConfig{buffered: true}, func(cache *Cache, resolver *client) {
+ resolver.domain <- "my.gitlab.com"
+
+ lookup, status, err := cache.Resolve(context.Background(), "my.gitlab.com")
+
+ assert.NoError(t, err)
+ assert.Equal(t, 200, status)
+ assert.Equal(t, "my.gitlab.com", lookup.Domain)
+ assert.Equal(t, uint64(1), resolver.resolutions)
+ })
+ })
+
+ t.Run("when item is not cached and accessed multiple times", func(t *testing.T) {
+ withTestCache(resolverConfig{}, func(cache *Cache, resolver *client) {
+ wg := &sync.WaitGroup{}
+ ctx := context.Background()
+
+ receiver := func() {
+ defer wg.Done()
+ cache.Resolve(ctx, "my.gitlab.com")
+ }
+
+ wg.Add(3)
+ go receiver()
+ go receiver()
+ go receiver()
+
+ assert.Equal(t, uint64(0), resolver.resolutions)
+
+ resolver.domain <- "my.gitlab.com"
+ wg.Wait()
+
+ assert.Equal(t, uint64(1), resolver.resolutions)
+ })
+ })
+
+ t.Run("when item is in short cache", func(t *testing.T) {
+ withTestCache(resolverConfig{}, func(cache *Cache, resolver *client) {
+ cache.withTestEntry(entryConfig{expired: false, retrieved: true}, func() {
+ lookup, _, _ := cache.Resolve(context.Background(), "my.gitlab.com")
+
+ assert.Equal(t, "my.gitlab.com", lookup.Domain)
+ assert.Equal(t, uint64(0), resolver.resolutions)
+ })
+ })
+ })
+
+ t.Run("when item is in long cache only", func(t *testing.T) {
+ withTestCache(resolverConfig{}, func(cache *Cache, resolver *client) {
+ cache.withTestEntry(entryConfig{expired: true, retrieved: true}, func() {
+ lookup, _, _ := cache.Resolve(context.Background(), "my.gitlab.com")
+
+ assert.Equal(t, "my.gitlab.com", lookup.Domain)
+ assert.Equal(t, uint64(0), resolver.resolutions)
+
+ resolver.domain <- "my.gitlab.com"
+ assert.Equal(t, uint64(1), resolver.resolutions)
+ })
+ })
+ })
+
+ t.Run("when item in long cache is requested multiple times", func(t *testing.T) {
+ withTestCache(resolverConfig{}, func(cache *Cache, resolver *client) {
+ cache.withTestEntry(entryConfig{expired: true, retrieved: true}, func() {
+ cache.Resolve(context.Background(), "my.gitlab.com")
+ cache.Resolve(context.Background(), "my.gitlab.com")
+ cache.Resolve(context.Background(), "my.gitlab.com")
+
+ assert.Equal(t, uint64(0), resolver.resolutions)
+
+ resolver.domain <- "my.gitlab.com"
+ assert.Equal(t, uint64(1), resolver.resolutions)
+ })
+ })
+ })
+
+ t.Run("when retrieval failed with an error", func(t *testing.T) {
+ // cache := NewCache()
+ // resolver := &stubbedClient{
+ // failure: errors.New("could not retrieve lookup"),
+ // }
+ //
+ // lookup := cache.GetLookup("my.gitlab.com", resolver.Resolve)
+ //
+ // assert.Equal(t, &Lookup{}, lookup)
+ })
+}
diff --git a/internal/source/gitlab/cache/entry.go b/internal/source/gitlab/cache/entry.go
new file mode 100644
index 00000000..65da4779
--- /dev/null
+++ b/internal/source/gitlab/cache/entry.go
@@ -0,0 +1,120 @@
+package cache
+
+import (
+ "context"
+ "sync"
+ "time"
+)
+
+var (
+ retrievalTimeout = 5 * time.Second
+ shortCacheExpiry = 10 * time.Second
+)
+
+// Entry represents a cache object that can be retrieved asynchronously and
+// holds a pointer to *Response when the domain lookup has been retrieved
+// succesfully
+type Entry struct {
+ domain string
+ created time.Time
+ ctx context.Context
+ cancel context.CancelFunc
+ fetch *sync.Once
+ refresh *sync.Once
+ mux *sync.RWMutex
+ retrieved chan struct{}
+ response *Response
+}
+
+func newCacheEntry(ctx context.Context, domain string) *Entry {
+ newctx, cancel := context.WithCancel(ctx)
+
+ return &Entry{
+ domain: domain,
+ ctx: newctx,
+ cancel: cancel,
+ created: time.Now(),
+ fetch: &sync.Once{},
+ refresh: &sync.Once{},
+ mux: &sync.RWMutex{},
+ retrieved: make(chan struct{}),
+ }
+}
+
+// IsUpToDate returns true if the entry has been resolved correctly and has not
+// expired yet. False otherwise.
+func (e *Entry) IsUpToDate() bool {
+ e.mux.RLock()
+ defer e.mux.RUnlock()
+
+ return e.isResolved() && !e.isExpired()
+}
+
+// NeedsRefresh return true if the entry has been resolved correctly but it has
+// expired since then.
+func (e *Entry) NeedsRefresh() bool {
+ e.mux.RLock()
+ defer e.mux.RUnlock()
+
+ return e.isResolved() && e.isExpired()
+}
+
+// Lookup return a retrieval response. TODO consider returning *Response.
+func (e *Entry) Lookup() (*Lookup, int, error) {
+ e.mux.RLock()
+ defer e.mux.RUnlock()
+
+ return e.response.Lookup()
+}
+
+// Retrieve schedules a retrieval of a response. It returns a channel that is
+// going to be closed when retrieval is done, either successfully or not.
+func (e *Entry) Retrieve(client Resolver) <-chan struct{} {
+ e.fetch.Do(func() {
+ retriever := Retriever{
+ client: client, ctx: e.ctx, timeout: retrievalTimeout,
+ }
+
+ go e.setResponse(retriever.Retrieve(e.domain))
+ })
+
+ return e.retrieved
+}
+
+// Refresh will update the entry in the store only when it gets resolved.
+func (e *Entry) Refresh(ctx context.Context, client Resolver, store Store) {
+ e.refresh.Do(func() {
+ go func() {
+ newEntry := newCacheEntry(ctx, e.domain)
+
+ <-newEntry.Retrieve(client)
+
+ store.ReplaceOrCreate(ctx, e.domain)
+ }()
+ })
+}
+
+// CancelContexts cancels all cancelable contexts. Typically used when the
+// entry is evicted from cache.
+func (e *Entry) CancelContexts() {
+ e.cancel()
+}
+
+func (e *Entry) setResponse(response <-chan Response) {
+ newResponse := <-response
+
+ e.mux.Lock()
+ defer e.mux.Unlock()
+
+ e.response = &newResponse
+
+ close(e.retrieved)
+}
+
+func (e *Entry) isExpired() bool {
+ return e.created.Add(shortCacheExpiry).Before(time.Now())
+}
+
+func (e *Entry) isResolved() bool {
+ return e.response != nil
+}
diff --git a/internal/source/gitlab/cache/lookup.go b/internal/source/gitlab/cache/lookup.go
new file mode 100644
index 00000000..e74c2e15
--- /dev/null
+++ b/internal/source/gitlab/cache/lookup.go
@@ -0,0 +1,19 @@
+package cache
+
+// Lookup defines a response that GitLab can send, which we can unmarshall
+type Lookup struct {
+ Domain string
+ CertificateCert string
+ CertificateKey string
+ // TODO prefix hash map
+ LookupPaths []struct {
+ Prefix string
+ ProjectID int
+ HTTPSOnly bool
+ AccessControl bool
+ Source struct {
+ Type string
+ Path string
+ }
+ }
+}
diff --git a/internal/source/gitlab/cache/memstore.go b/internal/source/gitlab/cache/memstore.go
new file mode 100644
index 00000000..2e66b217
--- /dev/null
+++ b/internal/source/gitlab/cache/memstore.go
@@ -0,0 +1,59 @@
+package cache
+
+import (
+ "context"
+ "sync"
+ "time"
+
+ cache "github.com/patrickmn/go-cache"
+)
+
+type memstore struct {
+ store *cache.Cache
+ mux *sync.Mutex
+}
+
+func newMemStore() Store {
+ memStore := &memstore{
+ // TODO onEvicted cancel context
+ store: cache.New(10*time.Minute, time.Minute),
+ mux: &sync.Mutex{},
+ }
+
+ memStore.store.OnEvicted(memStore.OnEvicted)
+
+ return memStore
+}
+
+func (m *memstore) LoadOrCreate(ctx context.Context, domain string) *Entry {
+ m.mux.Lock()
+ defer m.mux.Unlock()
+
+ if entry, exists := m.store.Get(domain); exists {
+ return entry.(*Entry)
+ }
+
+ entry := newCacheEntry(ctx, domain)
+ m.store.SetDefault(domain, entry)
+
+ return entry
+}
+
+func (m *memstore) ReplaceOrCreate(ctx context.Context, domain string) *Entry {
+ m.mux.Lock()
+ defer m.mux.Unlock()
+
+ entry := newCacheEntry(ctx, domain)
+
+ if _, exists := m.store.Get(domain); exists {
+ m.store.Delete(domain) // delete manually to trigger onEvicted
+ }
+
+ m.store.SetDefault(domain, entry)
+
+ return entry
+}
+
+func (m *memstore) OnEvicted(key string, value interface{}) {
+ value.(*Entry).CancelContexts()
+}
diff --git a/internal/source/gitlab/cache/resolver.go b/internal/source/gitlab/cache/resolver.go
new file mode 100644
index 00000000..7ae549dd
--- /dev/null
+++ b/internal/source/gitlab/cache/resolver.go
@@ -0,0 +1,8 @@
+package cache
+
+import "context"
+
+// Resolver represents an interface we use retrieve information from cache
+type Resolver interface {
+ Resolve(ctx context.Context, domain string) (*Lookup, int, error)
+}
diff --git a/internal/source/gitlab/cache/response.go b/internal/source/gitlab/cache/response.go
new file mode 100644
index 00000000..db4f611b
--- /dev/null
+++ b/internal/source/gitlab/cache/response.go
@@ -0,0 +1,13 @@
+package cache
+
+// Response represents a response sent by GitLab domains source.
+type Response struct {
+ lookup *Lookup
+ status int
+ err error
+}
+
+// Lookup is a helper method that returns all details of a Response
+func (r *Response) Lookup() (*Lookup, int, error) {
+ return r.lookup, r.status, r.err
+}
diff --git a/internal/source/gitlab/cache/retriever.go b/internal/source/gitlab/cache/retriever.go
new file mode 100644
index 00000000..914c3b4c
--- /dev/null
+++ b/internal/source/gitlab/cache/retriever.go
@@ -0,0 +1,49 @@
+package cache
+
+import (
+ "context"
+ "fmt"
+ "time"
+)
+
+// Retriever is an utility type that performs an HTTP request with backoff in
+// case of errors
+type Retriever struct {
+ client Resolver
+ ctx context.Context
+ timeout time.Duration
+}
+
+// Retrieve schedules a retrieval of a response and return a channel that the
+// response is going to be sent to
+func (r *Retriever) Retrieve(domain string) <-chan Response {
+ response := make(chan Response)
+
+ go r.retrieveWithBackoff(domain, response)
+
+ return response
+}
+
+func (r *Retriever) retrieveWithBackoff(domain string, response chan<- Response) {
+ newctx, cancel := context.WithTimeout(r.ctx, r.timeout)
+ defer cancel()
+
+ for i := 1; i <= 3; i++ {
+ lookup, status, err := r.client.Resolve(newctx, domain)
+ if err != nil {
+ time.Sleep(time.Second)
+ continue
+ }
+
+ select {
+ case <-newctx.Done():
+ fmt.Println("retrieval context done") // TODO logme
+ case response <- Response{lookup: lookup, status: status, err: err}:
+ fmt.Println("retrieval response sent") // TODO logme
+ }
+
+ break
+ }
+
+ close(response)
+}
diff --git a/internal/source/gitlab/cache/store.go b/internal/source/gitlab/cache/store.go
new file mode 100644
index 00000000..18ffb3ea
--- /dev/null
+++ b/internal/source/gitlab/cache/store.go
@@ -0,0 +1,9 @@
+package cache
+
+import "context"
+
+// Store defines an interface describing an abstract cache store
+type Store interface {
+ LoadOrCreate(ctx context.Context, domain string) *Entry
+ ReplaceOrCreate(ctx context.Context, domain string) *Entry
+}
diff --git a/internal/source/gitlab/cache_test.go b/internal/source/gitlab/cache_test.go
deleted file mode 100644
index b6c933ad..00000000
--- a/internal/source/gitlab/cache_test.go
+++ /dev/null
@@ -1,119 +0,0 @@
-package gitlab
-
-import (
- "context"
- "sync"
- "sync/atomic"
- "testing"
-
- "github.com/stretchr/testify/assert"
-)
-
-type stubbedLookup struct {
- resolutions uint64
- domainChan chan string
- failure error
-}
-
-func (s *stubbedLookup) Resolve(ctx context.Context) (Lookup, error) {
- atomic.AddUint64(&s.resolutions, 1)
-
- if s.failure != nil {
- return Lookup{}, s.failure
- }
-
- return Lookup{Domain: <-s.domainChan}, nil
-}
-
-func TestGetLookup(t *testing.T) {
- maxRetrievalInterval = 0
-
- t.Run("when item is not cached", func(t *testing.T) {
- cache := NewCache()
- resolver := &stubbedLookup{domainChan: make(chan string, 1)}
-
- resolver.domainChan <- "my.gitlab.com"
-
- lookup := cache.GetLookup("my.gitlab.com", resolver.Resolve)
-
- assert.Equal(t, "my.gitlab.com", lookup.Domain)
- assert.Equal(t, uint64(1), resolver.resolutions)
- })
-
- t.Run("when item is not cached and accessed multiple times", func(t *testing.T) {
- cache := NewCache()
- resolver := &stubbedLookup{domainChan: make(chan string)}
- wg := &sync.WaitGroup{}
-
- receiver := func() {
- defer wg.Done()
- cache.GetLookup("my.gitlab.com", resolver.Resolve)
- }
-
- wg.Add(3)
- go receiver()
- go receiver()
- go receiver()
-
- assert.Equal(t, uint64(0), resolver.resolutions)
-
- resolver.domainChan <- "my.gitlab.com"
- wg.Wait()
-
- assert.Equal(t, uint64(1), resolver.resolutions)
- })
-
- t.Run("when item is in short cache", func(t *testing.T) {
- cache := NewCache()
- resolver := &stubbedLookup{domainChan: make(chan string)}
- cache.store("my.gitlab.com", Lookup{Domain: "my.gitlab.com"})
-
- lookup := cache.GetLookup("my.gitlab.com", resolver.Resolve)
-
- assert.Equal(t, "my.gitlab.com", lookup.Domain)
- assert.Equal(t, uint64(0), resolver.resolutions)
- })
-
- t.Run("when item is in long cache only", func(t *testing.T) {
- cache := NewCache()
- resolver := &stubbedLookup{domainChan: make(chan string)}
- cache.store("my.gitlab.com", Lookup{Domain: "my.gitlab.com"})
- cache.shortCache.Delete("my.gitlab.com")
-
- lookup := cache.GetLookup("my.gitlab.com", resolver.Resolve)
-
- assert.Equal(t, "my.gitlab.com", lookup.Domain)
- assert.Equal(t, uint64(0), resolver.resolutions)
-
- resolver.domainChan <- "my.gitlab.com"
- assert.Equal(t, uint64(1), resolver.resolutions)
- })
-
- t.Run("when item in long cache is requested multiple times", func(t *testing.T) {
- cache := NewCache()
- resolver := &stubbedLookup{domainChan: make(chan string)}
- cache.store("my.gitlab.com", Lookup{Domain: "my.gitlab.com"})
- cache.shortCache.Delete("my.gitlab.com")
-
- lookup := cache.GetLookup("my.gitlab.com", resolver.Resolve)
- cache.GetLookup("my.gitlab.com", resolver.Resolve)
- cache.GetLookup("my.gitlab.com", resolver.Resolve)
-
- assert.Equal(t, "my.gitlab.com", lookup.Domain)
- assert.Equal(t, uint64(0), resolver.resolutions)
-
- resolver.domainChan <- "my.gitlab.com"
- assert.Equal(t, uint64(1), resolver.resolutions)
- })
-
- t.Run("when retrieval failed with an error", func(t *testing.T) {
- // cache := NewCache()
- // resolver := &stubbedLookup{
- // failure: errors.New("could not retrieve lookup"),
- // }
- //
- // lookup := cache.GetLookup("my.gitlab.com", resolver.Resolve)
- //
- // assert.Equal(t, &Lookup{}, lookup)
- })
-}
diff --git a/internal/source/gitlab/client.go b/internal/source/gitlab/client.go
index 5ec45d9d..b2dab32e 100644
--- a/internal/source/gitlab/client.go
+++ b/internal/source/gitlab/client.go
@@ -1,7 +1,9 @@
package gitlab
+import "context"
+
// Client is an internal HTTP client used for communication with GitLab
// instance
type Client interface {
- Resolve(domain string) *Lookup
+ Resolve(ctx context.Context, domain string) (*Lookup, int, error)
}
diff --git a/internal/source/gitlab/gitlab.go b/internal/source/gitlab/gitlab.go
index 8ac8fd85..7f67e1a0 100644
--- a/internal/source/gitlab/gitlab.go
+++ b/internal/source/gitlab/gitlab.go
@@ -5,13 +5,14 @@ import (
"gitlab.com/gitlab-org/gitlab-pages/internal/domain"
"gitlab.com/gitlab-org/gitlab-pages/internal/serving"
+ "gitlab.com/gitlab-org/gitlab-pages/internal/source/gitlab/cache"
)
// Gitlab source represent a new domains configuration source. We fetch all the
// information about domains from GitLab instance.
type Gitlab struct {
client Client
- cache Cache
+ cache cache.Cache
}
// GetDomain return a representation of a domain that we have fetched from
diff --git a/internal/source/gitlab/lookup.go b/internal/source/gitlab/lookup.go
index f6261276..a32f78d0 100644
--- a/internal/source/gitlab/lookup.go
+++ b/internal/source/gitlab/lookup.go
@@ -1,19 +1,5 @@
package gitlab
-// Lookup defines a response that GitLab can send, which we can unmarshall
+// Lookup is a struct that we need to TODO refactor
type Lookup struct {
- Domain string
- CertificateCert string
- CertificateKey string
- // TODO prefix hash map
- LookupPaths []struct {
- Prefix string
- ProjectID int
- HTTPSOnly bool
- AccessControl bool
- Source struct {
- Type string
- Path string
- }
- }
}