Introduce race detector and coverage profile (#149)

* Introduce race checker and converage profile
* Fix failed test cases

Signed-off-by: Gao Hongtao <hanahmily@gmail.com>
diff --git a/.github/workflows/go.yml b/.github/workflows/go.yml
index 7476539..ac275ce 100644
--- a/.github/workflows/go.yml
+++ b/.github/workflows/go.yml
@@ -115,10 +115,12 @@
       - name: Update dependencies 
         if: steps.cache-go.outputs.cache-hit != 'true'
         run: GOPROXY=https://proxy.golang.org go mod download
-      - name: Generate codes
+      - name: Generate mocks
         run: make generate
       - name: Test
-        run: make test
+        run: make test-ci
+      - name: Upload coverage to Codecov
+        uses: codecov/codecov-action@v3
   result:
     name: Continuous Integration
     runs-on: ubuntu-20.04
diff --git a/Makefile b/Makefile
index fbe9237..c888fdb 100644
--- a/Makefile
+++ b/Makefile
@@ -54,11 +54,18 @@
 test: default          ## Run the unit tests in all projects
 
 test-race: TARGET=test-race
+test-race: PROJECTS:=$(PROJECTS) pkg
 test-race: default     ## Run the unit tests in all projects with race detector on
 
 test-coverage: TARGET=test-coverage
+test-coverage: PROJECTS:=$(PROJECTS) pkg
 test-coverage: default ## Run the unit tests in all projects with coverage analysis on
 
+include scripts/build/ginkgo.mk
+
+test-ci: $(GINKGO) ## Run the unit tests in CI
+	$(GINKGO) --skip-package=test/stress --race --cover --covermode atomic --coverprofile=coverage.out ./... 
+
 ##@ Code quality targets
 
 lint: TARGET=lint
@@ -164,6 +171,6 @@
 
 .PHONY: all $(PROJECTS) clean build  default nuke
 .PHONY: lint check tidy format pre-push
-.PHONY: test test-race test-coverage 
+.PHONY: test test-race test-coverage test-ci
 .PHONY: license-check license-fix license-dep
 .PHONY: release release-binary release-source release-sign release-assembly
diff --git a/banyand/.gitignore b/banyand/.gitignore
new file mode 100644
index 0000000..8e705cf
--- /dev/null
+++ b/banyand/.gitignore
@@ -0,0 +1,19 @@
+# Licensed to Apache Software Foundation (ASF) under one or more contributor
+# license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright
+# ownership. Apache Software Foundation (ASF) licenses this file to you under
+# the Apache License, Version 2.0 (the "License"); you may
+# not use this file except in compliance with the License.
+# You may obtain a copy of the License at
+#
+#     http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing,
+# software distributed under the License is distributed on an
+# "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+# KIND, either express or implied.  See the License for the
+# specific language governing permissions and limitations
+# under the License.
+
+# Binaries for programs and plugins
+build
diff --git a/banyand/internal/cmd/standalone.go b/banyand/internal/cmd/standalone.go
index a4c292f..02f625c 100644
--- a/banyand/internal/cmd/standalone.go
+++ b/banyand/internal/cmd/standalone.go
@@ -40,7 +40,7 @@
 	"github.com/apache/skywalking-banyandb/pkg/version"
 )
 
-var g = run.Group{Name: "standalone"}
+var g = run.NewGroup("standalone")
 
 func newStandaloneCmd() *cobra.Command {
 	l := logger.GetLogger("bootstrap")
@@ -107,7 +107,7 @@
 			logger.GetLogger().Info().Msg("starting as a standalone server")
 			// Spawn our go routines and wait for shutdown.
 			if err := g.Run(); err != nil {
-				logger.GetLogger().Error().Err(err).Stack().Str("name", g.Name).Msg("Exit")
+				logger.GetLogger().Error().Err(err).Stack().Str("name", g.Name()).Msg("Exit")
 				os.Exit(-1)
 			}
 			return nil
diff --git a/banyand/liaison/grpc/grpc_suite_test.go b/banyand/liaison/grpc/grpc_suite_test.go
index 2596f41..281f3cf 100644
--- a/banyand/liaison/grpc/grpc_suite_test.go
+++ b/banyand/liaison/grpc/grpc_suite_test.go
@@ -27,7 +27,7 @@
 	"github.com/apache/skywalking-banyandb/pkg/logger"
 )
 
-var defaultEventallyTimeout = 30 * time.Second
+var defaultEventuallyTimeout = 30 * time.Second
 
 func TestGrpc(t *testing.T) {
 	RegisterFailHandler(Fail)
diff --git a/banyand/liaison/grpc/measure_test.go b/banyand/liaison/grpc/measure_test.go
index 321cbfd..299a1dd 100644
--- a/banyand/liaison/grpc/measure_test.go
+++ b/banyand/liaison/grpc/measure_test.go
@@ -64,7 +64,7 @@
 		measureWrite(conn)
 		Eventually(func() (int, error) {
 			return measureQuery(conn)
-		}).Should(Equal(1))
+		}, defaultEventuallyTimeout).Should(Equal(1))
 		_ = conn.Close()
 		gracefulStop()
 		By("Verifying an existing server")
@@ -78,7 +78,7 @@
 				return 0
 			}
 			return num
-		}, defaultEventallyTimeout).Should(Equal(1))
+		}, defaultEventuallyTimeout).Should(Equal(1))
 	})
 	It("is a TLS server", func() {
 		flags := []string{
@@ -103,7 +103,7 @@
 		measureWrite(conn)
 		Eventually(func() (int, error) {
 			return measureQuery(conn)
-		}, defaultEventallyTimeout).Should(Equal(1))
+		}, defaultEventuallyTimeout).Should(Equal(1))
 	})
 	AfterEach(func() {
 		_ = conn.Close()
diff --git a/banyand/liaison/grpc/stream_test.go b/banyand/liaison/grpc/stream_test.go
index a959e6f..3f3d3ff 100644
--- a/banyand/liaison/grpc/stream_test.go
+++ b/banyand/liaison/grpc/stream_test.go
@@ -84,7 +84,7 @@
 				return 0
 			}
 			return num
-		}, defaultEventallyTimeout).Should(Equal(1))
+		}, defaultEventuallyTimeout).Should(Equal(1))
 	})
 	It("is a TLS server", func() {
 		flags := []string{
@@ -108,7 +108,7 @@
 		streamWrite(conn)
 		Eventually(func() (int, error) {
 			return streamQuery(conn)
-		}, defaultEventallyTimeout).Should(Equal(1))
+		}, defaultEventuallyTimeout).Should(Equal(1))
 	})
 	AfterEach(func() {
 		_ = conn.Close()
diff --git a/banyand/measure/metadata_test.go b/banyand/measure/metadata_test.go
index 347b4e2..a3c5295 100644
--- a/banyand/measure/metadata_test.go
+++ b/banyand/measure/metadata_test.go
@@ -27,6 +27,7 @@
 	"github.com/apache/skywalking-banyandb/api/event"
 	commonv1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/common/v1"
 	databasev1 "github.com/apache/skywalking-banyandb/api/proto/banyandb/database/v1"
+	"github.com/apache/skywalking-banyandb/banyand/measure"
 	"github.com/apache/skywalking-banyandb/pkg/test"
 )
 
@@ -98,13 +99,13 @@
 			})
 			Expect(err).ShouldNot(HaveOccurred())
 			Expect(deleted).Should(BeTrue())
-			Eventually(func() bool {
+			Eventually(func() error {
 				_, err := svcs.measure.Measure(&commonv1.Metadata{
 					Name:  "service_cpm_minute",
 					Group: "sw_metric",
 				})
-				return err != nil
-			}).WithTimeout(10 * time.Second).Should(BeFalse())
+				return err
+			}).WithTimeout(30 * time.Second).Should(MatchError(measure.ErrMeasureNotExist))
 		})
 
 		Context("Update a measure", func() {
diff --git a/banyand/metadata/schema/etcd.go b/banyand/metadata/schema/etcd.go
index 3edc3ad..15f8eec 100644
--- a/banyand/metadata/schema/etcd.go
+++ b/banyand/metadata/schema/etcd.go
@@ -22,6 +22,7 @@
 	"net/url"
 	"os"
 	"path/filepath"
+	"sync"
 
 	"github.com/pkg/errors"
 	clientv3 "go.etcd.io/etcd/client/v3"
@@ -84,6 +85,7 @@
 	server   *embed.Etcd
 	kv       clientv3.KV
 	handlers []*eventHandler
+	mux      sync.RWMutex
 }
 
 type etcdSchemaRegistryConfig struct {
@@ -98,6 +100,8 @@
 }
 
 func (e *etcdSchemaRegistry) RegisterHandler(kind Kind, handler EventHandler) {
+	e.mux.Lock()
+	defer e.mux.Unlock()
 	e.handlers = append(e.handlers, &eventHandler{
 		interestKeys: kind,
 		handler:      handler,
@@ -105,7 +109,10 @@
 }
 
 func (e *etcdSchemaRegistry) notifyUpdate(metadata Metadata) {
-	for _, h := range e.handlers {
+	e.mux.RLock()
+	hh := e.handlers
+	e.mux.RUnlock()
+	for _, h := range hh {
 		if h.InterestOf(metadata.Kind) {
 			h.handler.OnAddOrUpdate(metadata)
 		}
@@ -113,7 +120,10 @@
 }
 
 func (e *etcdSchemaRegistry) notifyDelete(metadata Metadata) {
-	for _, h := range e.handlers {
+	e.mux.RLock()
+	hh := e.handlers
+	e.mux.RUnlock()
+	for _, h := range hh {
 		if h.InterestOf(metadata.Kind) {
 			h.handler.OnDelete(metadata)
 		}
diff --git a/banyand/tsdb/block.go b/banyand/tsdb/block.go
index 1ef6c3f..f913ce2 100644
--- a/banyand/tsdb/block.go
+++ b/banyand/tsdb/block.go
@@ -37,6 +37,7 @@
 	"github.com/apache/skywalking-banyandb/pkg/index/inverted"
 	"github.com/apache/skywalking-banyandb/pkg/index/lsm"
 	"github.com/apache/skywalking-banyandb/pkg/logger"
+	"github.com/apache/skywalking-banyandb/pkg/run"
 	"github.com/apache/skywalking-banyandb/pkg/timestamp"
 )
 
@@ -66,7 +67,8 @@
 	segID          uint16
 	blockID        uint16
 	encodingMethod EncodingMethod
-	flushCh        chan struct{}
+	flushCh        *run.Chan[struct{}]
+	flushChQueue   chan *run.Chan[struct{}]
 }
 
 type blockOpts struct {
@@ -101,11 +103,24 @@
 		Reporter:       bucket.NewTimeBasedReporter(timeRange, clock),
 		closed:         atomic.NewBool(true),
 		encodingMethod: encodingMethodObject.(EncodingMethod),
+		flushChQueue:   make(chan *run.Chan[struct{}]),
 	}
 	position := ctx.Value(common.PositionKey)
 	if position != nil {
 		b.position = position.(common.Position)
 	}
+	go func() {
+		for {
+			ch := <-b.flushChQueue
+			for {
+				_, more := ch.Read()
+				if !more {
+					break
+				}
+				b.flush()
+			}
+		}
+	}()
 	return b, err
 }
 
@@ -116,23 +131,15 @@
 		return nil
 	}
 	b.ref = z.NewCloser(1)
-	b.flushCh = make(chan struct{})
-	go func() {
-		for {
-			_, more := <-b.flushCh
-			if !more {
-				return
-			}
-			b.flush()
-		}
-	}()
+	b.flushCh = run.NewChan(make(chan struct{}))
+	b.flushChQueue <- b.flushCh
 	if b.store, err = kv.OpenTimeSeriesStore(
 		0,
 		path.Join(b.path, componentMain),
 		kv.TSSWithEncoding(b.encodingMethod.EncoderPool, b.encodingMethod.DecoderPool),
 		kv.TSSWithLogger(b.l.Named(componentMain)),
 		kv.TSSWithFlushCallback(func() {
-			b.flushCh <- struct{}{}
+			b.flushCh.Write(struct{}{})
 		}),
 	); err != nil {
 		return err
@@ -203,7 +210,7 @@
 		_ = closer.Close()
 	}
 	b.closed.Store(true)
-	close(b.flushCh)
+	b.flushCh.Close()
 }
 
 func (b *block) isClosed() bool {
diff --git a/banyand/tsdb/bucket/strategy.go b/banyand/tsdb/bucket/strategy.go
index e31cb74..3cd6531 100644
--- a/banyand/tsdb/bucket/strategy.go
+++ b/banyand/tsdb/bucket/strategy.go
@@ -18,6 +18,8 @@
 package bucket
 
 import (
+	"sync"
+
 	"github.com/pkg/errors"
 	"go.uber.org/multierr"
 
@@ -37,6 +39,7 @@
 	ctrl       Controller
 	current    Reporter
 	next       Reporter
+	mux        sync.Mutex
 	logger     *logger.Logger
 	stopCh     chan struct{}
 }
@@ -111,9 +114,11 @@
 					}
 				}
 				if ratio >= 1.0 {
+					s.mux.Lock()
 					s.ctrl.OnMove(s.current, s.next)
 					s.current = s.next
 					s.next = nil
+					s.mux.Unlock()
 					goto bucket
 				}
 			case <-s.stopCh:
@@ -124,6 +129,8 @@
 }
 
 func (s *Strategy) Close() {
-	s.ctrl.OnMove(s.current, nil)
 	close(s.stopCh)
+	s.mux.Lock()
+	defer s.mux.Unlock()
+	s.ctrl.OnMove(s.current, nil)
 }
diff --git a/banyand/tsdb/index/writer.go b/banyand/tsdb/index/writer.go
index ae9fde2..3f94c25 100644
--- a/banyand/tsdb/index/writer.go
+++ b/banyand/tsdb/index/writer.go
@@ -33,6 +33,7 @@
 	"github.com/apache/skywalking-banyandb/pkg/logger"
 	"github.com/apache/skywalking-banyandb/pkg/partition"
 	pbv1 "github.com/apache/skywalking-banyandb/pkg/pb/v1"
+	"github.com/apache/skywalking-banyandb/pkg/run"
 )
 
 type CallbackFn func()
@@ -61,7 +62,7 @@
 	l              *logger.Logger
 	db             tsdb.Supplier
 	shardNum       uint32
-	ch             chan Message
+	ch             *run.Chan[Message]
 	indexRuleIndex []*partition.IndexRuleLocator
 }
 
@@ -76,31 +77,25 @@
 	w.shardNum = options.ShardNum
 	w.db = options.DB
 	w.indexRuleIndex = partition.ParseIndexRuleLocators(options.Families, options.IndexRules)
-	w.ch = make(chan Message)
+	w.ch = run.NewChan[Message](make(chan Message))
 	w.bootIndexGenerator()
 	return w
 }
 
 func (s *Writer) Write(value Message) {
 	go func(m Message) {
-		defer func() {
-			if recover() != nil {
-				_ = m.BlockCloser.Close()
-			}
-		}()
-		s.ch <- m
+		s.ch.Write(m)
 	}(value)
 }
 
 func (s *Writer) Close() error {
-	close(s.ch)
-	return nil
+	return s.ch.Close()
 }
 
 func (s *Writer) bootIndexGenerator() {
 	go func() {
 		for {
-			m, more := <-s.ch
+			m, more := s.ch.Read()
 			if !more {
 				return
 			}
diff --git a/pkg/index/inverted/inverted.go b/pkg/index/inverted/inverted.go
index c126a6b..61b13c5 100644
--- a/pkg/index/inverted/inverted.go
+++ b/pkg/index/inverted/inverted.go
@@ -41,6 +41,7 @@
 	memTable          *memTable
 	immutableMemTable *memTable
 	rwMutex           sync.RWMutex
+	closed            bool
 
 	l *logger.Logger
 }
@@ -63,7 +64,7 @@
 }
 
 func (s *store) Close() error {
-	return s.diskTable.Close()
+	return s.flush(true)
 }
 
 func (s *store) Write(field index.Field, chunkID common.ItemID) error {
@@ -71,12 +72,22 @@
 }
 
 func (s *store) Flush() error {
+	return s.flush(false)
+}
+
+func (s *store) flush(toClose bool) error {
+	s.rwMutex.Lock()
+	defer func() {
+		if toClose && !s.closed {
+			_ = s.diskTable.Close()
+			s.closed = true
+		}
+		s.rwMutex.Unlock()
+	}()
 	state := s.memTable.Stats()
 	if state.MemBytes <= 0 {
 		return nil
 	}
-	s.rwMutex.Lock()
-	defer s.rwMutex.Unlock()
 	if s.immutableMemTable == nil {
 		s.immutableMemTable = s.memTable
 		s.memTable = newMemTable()
diff --git a/pkg/run/channel.go b/pkg/run/channel.go
new file mode 100644
index 0000000..82a9299
--- /dev/null
+++ b/pkg/run/channel.go
@@ -0,0 +1,50 @@
+// Licensed to Apache Software Foundation (ASF) under one or more contributor
+// license agreements. See the NOTICE file distributed with
+// this work for additional information regarding copyright
+// ownership. Apache Software Foundation (ASF) licenses this file to you under
+// the Apache License, Version 2.0 (the "License"); you may
+// not use this file except in compliance with the License.
+// You may obtain a copy of the License at
+//
+//     http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+package run
+
+import (
+	"sync"
+)
+
+type Chan[T any] struct {
+	ch     chan T
+	closer sync.WaitGroup
+}
+
+func NewChan[T any](ch chan T) *Chan[T] {
+	return &Chan[T]{
+		ch: ch,
+	}
+}
+
+func (c *Chan[T]) Write(item T) {
+	c.closer.Add(1)
+	defer c.closer.Done()
+	c.ch <- item
+}
+
+func (c *Chan[T]) Read() (T, bool) {
+	item, more := <-c.ch
+	return item, more
+}
+
+func (c *Chan[T]) Close() error {
+	c.closer.Wait()
+	close(c.ch)
+	return nil
+}
diff --git a/pkg/run/run.go b/pkg/run/run.go
index d4ba959..791e33e 100644
--- a/pkg/run/run.go
+++ b/pkg/run/run.go
@@ -22,7 +22,6 @@
 	"os"
 	"path"
 	"sync"
-	"time"
 
 	"github.com/oklog/run"
 	"github.com/pkg/errors"
@@ -119,23 +118,33 @@
 // to manage service lifecycles. It allows for easy composition of elegant
 // monoliths as well as adding signal handlers, metrics services, etc.
 type Group struct {
-	// Name of the Group managed service. If omitted, the binaryname will be
-	// used as found at runtime.
-	Name string
+	name string
 
-	f   *FlagSet
-	r   run.Group
-	c   []Config
-	p   []PreRunner
-	s   []Service
-	swg *sync.WaitGroup
-	log *logger.Logger
+	f       *FlagSet
+	r       run.Group
+	c       []Config
+	p       []PreRunner
+	s       []Service
+	readyCh chan struct{}
+	log     *logger.Logger
 
 	showRunGroup bool
 
 	configured bool
 }
 
+func NewGroup(name string) Group {
+	return Group{
+		name:    name,
+		readyCh: make(chan struct{}),
+	}
+}
+
+// Name shows the name of the group
+func (g Group) Name() string {
+	return g.name
+}
+
 // Register will inspect the provided objects implementing the Unit interface to
 // see if it needs to register the objects for any of the Group bootstrap
 // phases. If a Unit doesn't satisfy any of the bootstrap phases it is ignored
@@ -144,7 +153,7 @@
 // Units, signalling for each provided Unit if it successfully registered with
 // Group for at least one of the bootstrap phases or if it was ignored.
 func (g *Group) Register(units ...Unit) []bool {
-	g.log = logger.GetLogger(g.Name)
+	g.log = logger.GetLogger(g.name)
 	hasRegistered := make([]bool, len(units))
 	for idx := range units {
 		if !g.configured {
@@ -169,7 +178,7 @@
 
 func (g *Group) RegisterFlags() *FlagSet {
 	// run configuration stage
-	g.f = NewFlagSet(g.Name)
+	g.f = NewFlagSet(g.name)
 	g.f.SortFlags = false // keep order of flag registration
 	g.f.Usage = func() {
 		fmt.Printf("Flags:\n")
@@ -178,7 +187,7 @@
 
 	gFS := NewFlagSet("Common Service options")
 	gFS.SortFlags = false
-	gFS.StringVarP(&g.Name, "name", "n", g.Name, `name of this service`)
+	gFS.StringVarP(&g.name, "name", "n", g.name, `name of this service`)
 	gFS.BoolVar(&g.showRunGroup, "show-rungroup-units", false, "show rungroup units")
 	g.f.AddFlagSet(gFS.FlagSet)
 
@@ -216,12 +225,12 @@
 // If an error is returned the application must shut down as it is considered
 // fatal.
 func (g *Group) RunConfig() (interrupted bool, err error) {
-	g.log = logger.GetLogger(g.Name)
+	g.log = logger.GetLogger(g.name)
 	g.configured = true
 
-	if g.Name == "" {
+	if g.name == "" {
 		// use the binary name if custom name has not been provided
-		g.Name = path.Base(os.Args[0])
+		g.name = path.Base(os.Args[0])
 	}
 
 	defer func() {
@@ -314,8 +323,12 @@
 		}
 	}
 
-	g.swg = &sync.WaitGroup{}
-	g.swg.Add(len(g.s))
+	swg := &sync.WaitGroup{}
+	swg.Add(len(g.s))
+	go func() {
+		swg.Wait()
+		close(g.readyCh)
+	}()
 	// feed our registered services to our internal run.Group
 	for idx := range g.s {
 		// a Service might have been deregistered during Run
@@ -327,7 +340,7 @@
 		g.log.Debug().Uint32("total", uint32(len(g.s))).Uint32("ran", uint32(idx+1)).Str("name", s.Name()).Msg("serve")
 		g.r.Add(func() error {
 			notify := s.Serve()
-			g.swg.Done()
+			swg.Done()
 			<-notify
 			return nil
 		}, func(_ error) {
@@ -374,12 +387,9 @@
 		}
 	}
 
-	return fmt.Sprintf("Group: %s [%s]%s", g.Name, t, s)
+	return fmt.Sprintf("Group: %s [%s]%s", g.name, t, s)
 }
 
 func (g *Group) WaitTillReady() {
-	for g.swg == nil {
-		time.Sleep(100 * time.Microsecond)
-	}
-	g.swg.Wait()
+	<-g.readyCh
 }
diff --git a/pkg/schema/metadata.go b/pkg/schema/metadata.go
index 67a87c2..7e92376 100644
--- a/pkg/schema/metadata.go
+++ b/pkg/schema/metadata.go
@@ -325,11 +325,15 @@
 }
 
 func (sr *schemaRepo) NotifyAll() (err error) {
-	for _, g := range sr.getMap() {
+	sr.RLock()
+	defer sr.RUnlock()
+	for _, g := range sr.data {
 		err = multierr.Append(err, sr.notify(g.groupSchema, databasev1.Action_ACTION_PUT))
-		for _, s := range g.getMap() {
+		g.mapMutex.RLock()
+		for _, s := range g.schemaMap {
 			err = multierr.Append(err, g.notify(s, databasev1.Action_ACTION_PUT))
 		}
+		g.mapMutex.RUnlock()
 	}
 	return err
 }
@@ -346,7 +350,10 @@
 	if sr.workerStopCh != nil {
 		close(sr.workerStopCh)
 	}
-	for _, g := range sr.getMap() {
+
+	sr.RLock()
+	defer sr.RUnlock()
+	for _, g := range sr.data {
 		err := g.close()
 		if err != nil {
 			sr.l.Err(err).Stringer("group", g.groupSchema.Metadata).Msg("closing")
@@ -354,12 +361,6 @@
 	}
 }
 
-func (sr *schemaRepo) getMap() map[string]*group {
-	sr.RLock()
-	defer sr.RUnlock()
-	return sr.data
-}
-
 var _ Group = (*group)(nil)
 
 type group struct {
@@ -469,8 +470,9 @@
 }
 
 func (g *group) LoadResource(name string) (Resource, bool) {
-	data := g.getMap()
-	s := data[name]
+	g.mapMutex.RLock()
+	s := g.schemaMap[name]
+	g.mapMutex.RUnlock()
 	if s == nil {
 		return nil, false
 	}
@@ -501,14 +503,10 @@
 }
 
 func (g *group) close() (err error) {
-	for _, s := range g.getMap() {
+	g.mapMutex.RLock()
+	for _, s := range g.schemaMap {
 		err = multierr.Append(err, s.Close())
 	}
+	g.mapMutex.RUnlock()
 	return multierr.Append(err, g.SupplyTSDB().Close())
 }
-
-func (g *group) getMap() map[string]Resource {
-	g.mapMutex.RLock()
-	defer g.mapMutex.RUnlock()
-	return g.schemaMap
-}
diff --git a/pkg/test/setup.go b/pkg/test/setup.go
index dfee987..5dc4f56 100644
--- a/pkg/test/setup.go
+++ b/pkg/test/setup.go
@@ -134,7 +134,7 @@
 
 func SetUpModules(flags []string, units ...run.Unit) func() {
 	closer := run.NewTester("closer")
-	g := run.Group{Name: "standalone-test"}
+	g := run.NewGroup("standalone-test")
 	g.Register(append([]run.Unit{closer}, units...)...)
 	err := g.RegisterFlags().Parse(flags)
 	gomega.Expect(err).NotTo(gomega.HaveOccurred())
diff --git a/scripts/build/ginkgo.mk b/scripts/build/ginkgo.mk
new file mode 100644
index 0000000..b3152b6
--- /dev/null
+++ b/scripts/build/ginkgo.mk
@@ -0,0 +1,5 @@
+GINKGO := $(tool_bin)/ginkgo
+$(GINKGO):
+	@echo "Install ginkgo..."
+	@mkdir -p $(tool_bin)
+	@GOBIN=$(tool_bin) go install github.com/onsi/ginkgo/v2/ginkgo@v2.1.4
diff --git a/scripts/build/test.mk b/scripts/build/test.mk
index ad65bdf..ba9fcb1 100644
--- a/scripts/build/test.mk
+++ b/scripts/build/test.mk
@@ -22,19 +22,16 @@
 TEST_PKG_LIST ?= ./...
 
 TEST_COVERAGE_DIR ?= build/coverage
-TEST_COVERAGE_PROFILE := $(TEST_COVERAGE_DIR)/coverage.out
+TEST_COVERAGE_PROFILE_NAME := coverprofile.out
+TEST_COVERAGE_PROFILE := $(TEST_COVERAGE_DIR)/$(TEST_COVERAGE_PROFILE_NAME)
 TEST_COVERAGE_REPORT := $(TEST_COVERAGE_DIR)/coverage.html
 TEST_COVERAGE_PKG_LIST ?= $(TEST_PKG_LIST)
-TEST_COVERAGE_OPTS ?= -covermode=atomic -coverpkg=./...
+TEST_COVERAGE_OPTS ?= --covermode atomic --coverpkg ./...
 TEST_COVERAGE_EXTRA_OPTS ?=
 
 ##@ Test targets
 
-GINKGO := $(tool_bin)/ginkgo
-$(GINKGO):
-	@echo "Install ginkgo..."
-	@mkdir -p $(tool_bin)
-	@GOBIN=$(tool_bin) go install github.com/onsi/ginkgo/v2/ginkgo@v2.1.4
+include $(root_dir)/scripts/build/ginkgo.mk
 
 .PHONY: test
 test: $(GINKGO) generate ## Run all the unit tests
@@ -47,7 +44,8 @@
 .PHONY: test-coverage
 test-coverage: $(GINKGO) generate ## Run all the unit tests with coverage analysis on
 	mkdir -p "$(TEST_COVERAGE_DIR)"
-	$(GINKGO) $(TEST_COVERAGE_OPTS) $(TEST_COVERAGE_EXTRA_OPTS) --coverprofile="$(TEST_COVERAGE_PROFILE)" --tags "$(TEST_TAGS)" $(TEST_COVERAGE_PKG_LIST)
+	$(GINKGO) --cover $(TEST_COVERAGE_OPTS) $(TEST_COVERAGE_EXTRA_OPTS)  --tags "$(TEST_TAGS)" $(TEST_COVERAGE_PKG_LIST) \
+	   && mv $(TEST_COVERAGE_PROFILE_NAME) $(TEST_COVERAGE_DIR)
 	go tool cover -html="$(TEST_COVERAGE_PROFILE)" -o "$(TEST_COVERAGE_REPORT)"
 	@echo "Test coverage report has been saved to $(TEST_COVERAGE_REPORT)"
 
diff --git a/ui/Makefile b/ui/Makefile
index 25d6159..8c24670 100644
--- a/ui/Makefile
+++ b/ui/Makefile
@@ -47,6 +47,14 @@
 test:
 	@echo "No test"
 
+.PHONY: test-race
+test-race:
+	@echo "No test"
+
+.PHONY: test-coverage
+test-coverage:
+	@echo "No test"
+
 .PHONY: release
 release: build