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

gitlab.com/gitlab-org/gitaly.git - Unnamed repository; edit this file 'description' to name the repository.
summaryrefslogtreecommitdiff
diff options
context:
space:
mode:
authorJustin Tobler <jtobler@gitlab.com>2023-02-24 01:45:08 +0300
committerJustin Tobler <jtobler@gitlab.com>2023-02-24 01:45:08 +0300
commitfb7ac6faa5b8e8cad4a66e597665eb12d398b84d (patch)
tree02ea0c8bce3afdd257b1369e196f28b5fbc046f3
parent6e31710a1c698c92b55fc90918278d8234677130 (diff)
parent0aa76e990f06374d7a882f9b525e7edceb0199ae (diff)
Merge branch 'qmnguyen0711/fix-git-cat-file-traces' into 'master'
Fix quirky problems with tracing See merge request https://gitlab.com/gitlab-org/gitaly/-/merge_requests/5406 Merged-by: Justin Tobler <jtobler@gitlab.com> Approved-by: Justin Tobler <jtobler@gitlab.com> Reviewed-by: Will Chandler <wchandler@gitlab.com> Co-authored-by: Quang-Minh Nguyen <qmnguyen@gitlab.com>
-rw-r--r--client/dial_test.go5
-rw-r--r--cmd/gitaly/main.go8
-rw-r--r--internal/command/command.go13
-rw-r--r--internal/command/spawntoken.go4
-rw-r--r--internal/git/catfile/cache.go9
-rw-r--r--internal/git/catfile/commit.go8
-rw-r--r--internal/git/catfile/tag.go4
-rw-r--r--internal/git/catfile/tracing.go3
-rw-r--r--internal/git/catfile/tree_entries.go12
-rw-r--r--internal/git/housekeeping/clean_stale_data.go4
-rw-r--r--internal/git/housekeeping/optimize_repository.go4
-rw-r--r--internal/gitaly/service/repository/license.go4
-rw-r--r--internal/middleware/limithandler/concurrency_limiter.go6
-rw-r--r--internal/middleware/limithandler/rate_limiter.go6
-rw-r--r--internal/sidechannel/sidechannel.go4
-rw-r--r--internal/tracing/noop.go74
-rw-r--r--internal/tracing/tracing.go45
-rw-r--r--internal/tracing/tracing_test.go152
18 files changed, 322 insertions, 43 deletions
diff --git a/client/dial_test.go b/client/dial_test.go
index bc05b1b75..b3d2186e0 100644
--- a/client/dial_test.go
+++ b/client/dial_test.go
@@ -20,6 +20,7 @@ import (
gitalyauth "gitlab.com/gitlab-org/gitaly/v15/auth"
internalclient "gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/client"
"gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
gitalyx509 "gitlab.com/gitlab-org/gitaly/v15/internal/x509"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
"gitlab.com/gitlab-org/labkit/correlation"
@@ -379,7 +380,7 @@ func TestDial_Tracing(t *testing.T) {
)
svc := &testSvc{
unaryCall: func(ctx context.Context, r *grpc_testing.SimpleRequest) (*grpc_testing.SimpleResponse, error) {
- span, _ := opentracing.StartSpanFromContext(ctx, "nested-span")
+ span, _ := tracing.StartSpan(ctx, "nested-span", nil)
defer span.Finish()
span.LogKV("was", "called")
return &grpc_testing.SimpleResponse{}, nil
@@ -393,7 +394,7 @@ func TestDial_Tracing(t *testing.T) {
return stream.Context().Err()
}
- span, _ := opentracing.StartSpanFromContext(stream.Context(), "nested-span")
+ span, _ := tracing.StartSpan(stream.Context(), "nested-span", nil)
defer span.Finish()
span.LogKV("was", "called")
return nil
diff --git a/cmd/gitaly/main.go b/cmd/gitaly/main.go
index 2577b1c02..acc068cac 100644
--- a/cmd/gitaly/main.go
+++ b/cmd/gitaly/main.go
@@ -9,7 +9,6 @@ import (
"time"
"github.com/go-enry/go-license-detector/v4/licensedb"
- "github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
log "github.com/sirupsen/logrus"
@@ -45,10 +44,11 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/middleware/limithandler"
"gitlab.com/gitlab-org/gitaly/v15/internal/streamcache"
"gitlab.com/gitlab-org/gitaly/v15/internal/tempdir"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/internal/version"
"gitlab.com/gitlab-org/labkit/fips"
"gitlab.com/gitlab-org/labkit/monitoring"
- "gitlab.com/gitlab-org/labkit/tracing"
+ labkittracing "gitlab.com/gitlab-org/labkit/tracing"
"google.golang.org/grpc"
)
@@ -126,7 +126,7 @@ func configure(configPath string) (config.Cfg, error) {
sentry.ConfigureSentry(version.GetVersion(), sentry.Config(cfg.Logging.Sentry))
cfg.Prometheus.Configure()
- tracing.Initialize(tracing.WithServiceName("gitaly"))
+ labkittracing.Initialize(labkittracing.WithServiceName("gitaly"))
preloadLicenseDatabase()
return cfg, nil
@@ -146,7 +146,7 @@ func run(cfg config.Cfg) error {
ctx, cancel := context.WithCancel(context.Background())
defer cancel()
- bootstrapSpan, ctx := opentracing.StartSpanFromContext(ctx, "gitaly-bootstrap")
+ bootstrapSpan, ctx := tracing.StartSpan(ctx, "gitaly-bootstrap", nil)
defer bootstrapSpan.Finish()
if cfg.RuntimeDir != "" {
diff --git a/internal/command/command.go b/internal/command/command.go
index 490faad40..3bae16807 100644
--- a/internal/command/command.go
+++ b/internal/command/command.go
@@ -21,7 +21,8 @@ import (
"github.com/sirupsen/logrus"
"gitlab.com/gitlab-org/gitaly/v15/internal/command/commandcounter"
"gitlab.com/gitlab-org/gitaly/v15/internal/metadata/featureflag"
- "gitlab.com/gitlab-org/labkit/tracing"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
+ labkittracing "gitlab.com/gitlab-org/labkit/tracing"
)
var (
@@ -114,7 +115,7 @@ var (
// envInjector is responsible for injecting environment variables required for tracing into
// the child process.
- envInjector = tracing.NewEnvInjector()
+ envInjector = labkittracing.NewEnvInjector()
)
const (
@@ -199,11 +200,13 @@ func New(ctx context.Context, nameAndArgs []string, opts ...Option) (*Command, e
} else {
spanName = cmdName
}
- span, ctx := opentracing.StartSpanFromContext(
+ span, ctx := tracing.StartSpanIfHasParent(
ctx,
spanName,
- opentracing.Tag{Key: "path", Value: nameAndArgs[0]},
- opentracing.Tag{Key: "args", Value: strings.Join(nameAndArgs[1:], " ")},
+ tracing.Tags{
+ "path": nameAndArgs[0],
+ "args": strings.Join(nameAndArgs[1:], " "),
+ },
)
cmd := exec.Command(nameAndArgs[0], nameAndArgs[1:]...)
diff --git a/internal/command/spawntoken.go b/internal/command/spawntoken.go
index 409507a4a..dbdec3da7 100644
--- a/internal/command/spawntoken.go
+++ b/internal/command/spawntoken.go
@@ -7,9 +7,9 @@ import (
"github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
"github.com/kelseyhightower/envconfig"
- "github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"github.com/prometheus/client_golang/prometheus/promauto"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
)
const logDurationThreshold = 5 * time.Millisecond
@@ -58,7 +58,7 @@ func getSpawnToken(ctx context.Context) (putToken func(), err error) {
// https://gitlab.com/gitlab-org/gitaly/issues/823.
start := time.Now()
- span, ctx := opentracing.StartSpanFromContext(ctx, "command.getSpawnToken")
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "command.getSpawnToken", nil)
defer span.Finish()
select {
diff --git a/internal/git/catfile/cache.go b/internal/git/catfile/cache.go
index 1e21c31ee..8c75de3dc 100644
--- a/internal/git/catfile/cache.go
+++ b/internal/git/catfile/cache.go
@@ -7,13 +7,13 @@ import (
"sync"
"time"
- "github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v15/internal/helper"
"gitlab.com/gitlab-org/gitaly/v15/internal/metadata"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/labkit/correlation"
)
@@ -212,7 +212,7 @@ func (c *ProcessCache) getOrCreateProcess(
) (_ cacheable, _ func(), returnedErr error) {
defer c.reportCacheMembers()
- span, ctx := opentracing.StartSpanFromContext(ctx, spanName)
+ span, ctx := tracing.StartSpanIfHasParent(ctx, spanName, nil)
defer span.Finish()
cacheKey, isCacheable := newCacheKey(metadata.GetValue(ctx, SessionIDField), repo)
@@ -234,6 +234,11 @@ func (c *ProcessCache) getOrCreateProcess(
c.catfileCacheCounter.WithLabelValues("miss").Inc()
span.SetTag("hit", false)
+ // When cache misses, a new process is created. This process may be re-used later.
+ // In that case, the lifecycle of the process is stretched across multiple
+ // gorountines. We should not attribute the span of this shared process to the
+ // current trace.
+ ctx = tracing.DiscardSpanInContext(ctx)
// We have not found any cached process, so we need to create a new one. In this
// case, we need to detach the process from the current context such that it does
// not get killed when the parent context is cancelled.
diff --git a/internal/git/catfile/commit.go b/internal/git/catfile/commit.go
index 3d59ceacc..4b1268963 100644
--- a/internal/git/catfile/commit.go
+++ b/internal/git/catfile/commit.go
@@ -7,16 +7,16 @@ import (
"fmt"
"io"
- "github.com/opentracing/opentracing-go"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/repository"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/trailerparser"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
// GetCommit looks up a commit by revision using an existing Batch instance.
func GetCommit(ctx context.Context, objectReader ObjectContentReader, revision git.Revision) (*gitalypb.GitCommit, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "catfile.GetCommit", opentracing.Tag{Key: "revision", Value: revision.String()})
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "catfile.GetCommit", tracing.Tags{"revision": revision.String()})
defer span.Finish()
object, err := objectReader.Object(ctx, revision+"^{commit}")
@@ -36,7 +36,7 @@ func GetCommitWithTrailers(
objectReader ObjectContentReader,
revision git.Revision,
) (*gitalypb.GitCommit, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "catfile.GetCommitWithTrailers", opentracing.Tag{Key: "revision", Value: revision.String()})
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "catfile.GetCommitWithTrailers", tracing.Tags{"revision": revision.String()})
defer span.Finish()
commit, err := GetCommit(ctx, objectReader, revision)
@@ -75,7 +75,7 @@ func GetCommitWithTrailers(
// GetCommitMessage looks up a commit message and returns it in its entirety.
func GetCommitMessage(ctx context.Context, objectReader ObjectContentReader, repo repository.GitRepo, revision git.Revision) ([]byte, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "catfile.GetCommitMessage", opentracing.Tag{Key: "revision", Value: revision.String()})
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "catfile.GetCommitMessage", tracing.Tags{"revision": revision.String()})
defer span.Finish()
obj, err := objectReader.Object(ctx, revision+"^{commit}")
diff --git a/internal/git/catfile/tag.go b/internal/git/catfile/tag.go
index dbc46a121..2db9d870f 100644
--- a/internal/git/catfile/tag.go
+++ b/internal/git/catfile/tag.go
@@ -6,9 +6,9 @@ import (
"fmt"
"io"
- "github.com/opentracing/opentracing-go"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/helper"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
@@ -16,7 +16,7 @@ import (
// in the tagName because the tag name from refs/tags may be different than the name found in the
// actual tag object. We want to use the tagName found in refs/tags
func GetTag(ctx context.Context, objectReader ObjectContentReader, tagID git.Revision, tagName string) (*gitalypb.Tag, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "catfile.GetTag", opentracing.Tag{Key: "tagName", Value: tagName})
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "catfile.GetTag", tracing.Tags{"tagName": tagName})
defer span.Finish()
object, err := objectReader.Object(ctx, tagID)
diff --git a/internal/git/catfile/tracing.go b/internal/git/catfile/tracing.go
index a1a6fd90b..845c5c037 100644
--- a/internal/git/catfile/tracing.go
+++ b/internal/git/catfile/tracing.go
@@ -6,6 +6,7 @@ import (
"github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
)
type trace struct {
@@ -24,7 +25,7 @@ func startTrace(
counter *prometheus.CounterVec,
methodName string,
) *trace {
- span, _ := opentracing.StartSpanFromContext(ctx, methodName)
+ span, _ := tracing.StartSpanIfHasParent(ctx, methodName, nil)
trace := &trace{
span: span,
diff --git a/internal/git/catfile/tree_entries.go b/internal/git/catfile/tree_entries.go
index 96b781641..87b66a236 100644
--- a/internal/git/catfile/tree_entries.go
+++ b/internal/git/catfile/tree_entries.go
@@ -11,8 +11,8 @@ import (
"path/filepath"
"strings"
- "github.com/opentracing/opentracing-go"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
@@ -36,11 +36,10 @@ func NewTreeEntryFinder(objectReader ObjectContentReader, objectInfoReader Objec
// FindByRevisionAndPath returns a TreeEntry struct for the object present at the revision/path pair.
func (tef *TreeEntryFinder) FindByRevisionAndPath(ctx context.Context, revision, path string) (*gitalypb.TreeEntry, error) {
- span, ctx := opentracing.StartSpanFromContext(
+ span, ctx := tracing.StartSpanIfHasParent(
ctx,
"catfile.FindByRevisionAndPatch",
- opentracing.Tag{Key: "revision", Value: revision},
- opentracing.Tag{Key: "path", Value: path},
+ tracing.Tags{"revision": revision, "path": path},
)
defer span.Finish()
@@ -120,11 +119,10 @@ func TreeEntries(
objectInfoReader ObjectInfoReader,
revision, path string,
) (_ []*gitalypb.TreeEntry, returnedErr error) {
- span, ctx := opentracing.StartSpanFromContext(
+ span, ctx := tracing.StartSpanIfHasParent(
ctx,
"catfile.TreeEntries",
- opentracing.Tag{Key: "revision", Value: revision},
- opentracing.Tag{Key: "path", Value: path},
+ tracing.Tags{"revision": revision, "path": path},
)
defer span.Finish()
diff --git a/internal/git/housekeeping/clean_stale_data.go b/internal/git/housekeeping/clean_stale_data.go
index f4e1303cb..7e27d82a1 100644
--- a/internal/git/housekeeping/clean_stale_data.go
+++ b/internal/git/housekeeping/clean_stale_data.go
@@ -12,13 +12,13 @@ import (
"time"
"github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
- "github.com/opentracing/opentracing-go"
log "github.com/sirupsen/logrus"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v15/internal/helper/perm"
"gitlab.com/gitlab-org/gitaly/v15/internal/safe"
"gitlab.com/gitlab-org/gitaly/v15/internal/structerr"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"google.golang.org/grpc/codes"
)
@@ -45,7 +45,7 @@ type staleFileFinderFn func(context.Context, string) ([]string, error)
// CleanStaleData cleans up any stale data in the repository.
func (m *RepositoryManager) CleanStaleData(ctx context.Context, repo *localrepo.Repo) error {
- span, ctx := opentracing.StartSpanFromContext(ctx, "housekeeping.CleanStaleData")
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "housekeeping.CleanStaleData", nil)
defer span.Finish()
repoPath, err := repo.Path()
diff --git a/internal/git/housekeeping/optimize_repository.go b/internal/git/housekeeping/optimize_repository.go
index ff9b32eb7..bce297315 100644
--- a/internal/git/housekeeping/optimize_repository.go
+++ b/internal/git/housekeeping/optimize_repository.go
@@ -7,11 +7,11 @@ import (
"strconv"
"github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
- "github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/stats"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
)
// OptimizeRepositoryConfig is the configuration used by OptimizeRepository that is computed by
@@ -43,7 +43,7 @@ func (m *RepositoryManager) OptimizeRepository(
repo *localrepo.Repo,
opts ...OptimizeRepositoryOption,
) error {
- span, ctx := opentracing.StartSpanFromContext(ctx, "housekeeping.OptimizeRepository")
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "housekeeping.OptimizeRepository", nil)
defer span.Finish()
path, err := repo.Path()
diff --git a/internal/gitaly/service/repository/license.go b/internal/gitaly/service/repository/license.go
index 2d8400977..42accce07 100644
--- a/internal/gitaly/service/repository/license.go
+++ b/internal/gitaly/service/repository/license.go
@@ -13,7 +13,6 @@ import (
"github.com/go-enry/go-license-detector/v4/licensedb"
"github.com/go-enry/go-license-detector/v4/licensedb/api"
"github.com/go-enry/go-license-detector/v4/licensedb/filer"
- "github.com/opentracing/opentracing-go"
"gitlab.com/gitlab-org/gitaly/v15/internal/git"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/localrepo"
"gitlab.com/gitlab-org/gitaly/v15/internal/git/lstree"
@@ -21,6 +20,7 @@ import (
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/service"
"gitlab.com/gitlab-org/gitaly/v15/internal/metadata/featureflag"
"gitlab.com/gitlab-org/gitaly/v15/internal/structerr"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/internal/unarycache"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
)
@@ -86,7 +86,7 @@ func (s *server) FindLicense(ctx context.Context, req *gitalypb.FindLicenseReque
}
func findLicense(ctx context.Context, repo *localrepo.Repo, commitID git.ObjectID) (*gitalypb.FindLicenseResponse, error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "repository.findLicense")
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "repository.findLicense", nil)
defer span.Finish()
repoFiler := &gitFiler{ctx: ctx, repo: repo, treeishID: commitID}
diff --git a/internal/middleware/limithandler/concurrency_limiter.go b/internal/middleware/limithandler/concurrency_limiter.go
index 4de23c691..d200ac8ae 100644
--- a/internal/middleware/limithandler/concurrency_limiter.go
+++ b/internal/middleware/limithandler/concurrency_limiter.go
@@ -8,11 +8,11 @@ import (
"time"
"github.com/grpc-ecosystem/go-grpc-middleware/logging/logrus/ctxlogrus"
- "github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v15/internal/helper"
"gitlab.com/gitlab-org/gitaly/v15/internal/structerr"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
"google.golang.org/protobuf/types/known/durationpb"
)
@@ -160,10 +160,10 @@ func NewConcurrencyLimiter(maxConcurrencyLimit, maxQueueLength int, maxQueuedTic
// semaphore. If this takes longer than the maximum queueing limit then the caller will be
// dequeued and gets an error.
func (c *ConcurrencyLimiter) Limit(ctx context.Context, limitingKey string, f LimitedFunc) (interface{}, error) {
- span, ctx := opentracing.StartSpanFromContext(
+ span, ctx := tracing.StartSpanIfHasParent(
ctx,
"limithandler.ConcurrencyLimiter.Limit",
- opentracing.Tag{Key: "key", Value: limitingKey},
+ tracing.Tags{"key": limitingKey},
)
defer span.Finish()
diff --git a/internal/middleware/limithandler/rate_limiter.go b/internal/middleware/limithandler/rate_limiter.go
index 68ac5513d..6a84727cd 100644
--- a/internal/middleware/limithandler/rate_limiter.go
+++ b/internal/middleware/limithandler/rate_limiter.go
@@ -6,11 +6,11 @@ import (
"sync"
"time"
- "github.com/opentracing/opentracing-go"
"github.com/prometheus/client_golang/prometheus"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/config"
"gitlab.com/gitlab-org/gitaly/v15/internal/helper"
"gitlab.com/gitlab-org/gitaly/v15/internal/structerr"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"gitlab.com/gitlab-org/gitaly/v15/proto/go/gitalypb"
"golang.org/x/time/rate"
"google.golang.org/protobuf/types/known/durationpb"
@@ -33,10 +33,10 @@ var ErrRateLimit = errors.New("rate limit reached")
// Limit rejects an incoming reequest if the maximum number of requests per
// second has been reached
func (r *RateLimiter) Limit(ctx context.Context, lockKey string, f LimitedFunc) (interface{}, error) {
- span, _ := opentracing.StartSpanFromContext(
+ span, _ := tracing.StartSpanIfHasParent(
ctx,
"limithandler.RateLimiterLimit",
- opentracing.Tag{Key: "key", Value: lockKey},
+ tracing.Tags{"key": lockKey},
)
defer span.Finish()
diff --git a/internal/sidechannel/sidechannel.go b/internal/sidechannel/sidechannel.go
index d29ab9420..4fd3c679a 100644
--- a/internal/sidechannel/sidechannel.go
+++ b/internal/sidechannel/sidechannel.go
@@ -9,11 +9,11 @@ import (
"strconv"
"time"
- "github.com/opentracing/opentracing-go"
"github.com/sirupsen/logrus"
"gitlab.com/gitlab-org/gitaly/v15/internal/backchannel"
"gitlab.com/gitlab-org/gitaly/v15/internal/gitaly/client"
"gitlab.com/gitlab-org/gitaly/v15/internal/listenmux"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/tracing"
"google.golang.org/grpc"
"google.golang.org/grpc/credentials"
"google.golang.org/grpc/credentials/insecure"
@@ -34,7 +34,7 @@ const (
// OpenSidechannel opens a sidechannel connection from the stream opener
// extracted from the current peer connection.
func OpenSidechannel(ctx context.Context) (_ *ServerConn, err error) {
- span, ctx := opentracing.StartSpanFromContext(ctx, "sidechannel.OpenSidechannel")
+ span, ctx := tracing.StartSpanIfHasParent(ctx, "sidechannel.OpenSidechannel", nil)
defer span.Finish()
md, ok := metadata.FromIncomingContext(ctx)
diff --git a/internal/tracing/noop.go b/internal/tracing/noop.go
new file mode 100644
index 000000000..38d96c76e
--- /dev/null
+++ b/internal/tracing/noop.go
@@ -0,0 +1,74 @@
+package tracing
+
+import (
+ "github.com/opentracing/opentracing-go"
+ "github.com/opentracing/opentracing-go/log"
+)
+
+// NoopSpan is a dummy span implementing opentracing.Span interface. All data setting functions do
+// nothing. Data getting functions return other dummy objects. Spans of this kind are not recorded
+// later.
+type NoopSpan struct{}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) Finish() {}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) FinishWithOptions(_ opentracing.FinishOptions) {}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) Context() opentracing.SpanContext { return NoopSpanContext{} }
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) LogFields(...log.Field) {}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) SetOperationName(string) opentracing.Span { return s }
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) Log(opentracing.LogData) {}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) SetTag(string, interface{}) opentracing.Span { return s }
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) LogKV(...interface{}) {}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) SetBaggageItem(string, string) opentracing.Span { return s }
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) BaggageItem(string) string { return "" }
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) Tracer() opentracing.Tracer { return &NoopTracer{} }
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) LogEvent(string) {}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (s *NoopSpan) LogEventWithPayload(string, interface{}) {}
+
+// NoopSpanContext is a dummy context returned by NoopSpan
+type NoopSpanContext struct{}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (n NoopSpanContext) ForeachBaggageItem(func(k string, v string) bool) {}
+
+// NoopTracer is a dummy tracer returned by NoopSpan
+type NoopTracer struct{}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (n NoopTracer) StartSpan(string, ...opentracing.StartSpanOption) opentracing.Span {
+ return &NoopSpan{}
+}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (n NoopTracer) Inject(opentracing.SpanContext, interface{}, interface{}) error {
+ return nil
+}
+
+//nolint:revive // This is unintentionally missing documentation.
+func (n NoopTracer) Extract(interface{}, interface{}) (opentracing.SpanContext, error) {
+ return &NoopSpanContext{}, nil
+}
diff --git a/internal/tracing/tracing.go b/internal/tracing/tracing.go
new file mode 100644
index 000000000..4ba7c2091
--- /dev/null
+++ b/internal/tracing/tracing.go
@@ -0,0 +1,45 @@
+package tracing
+
+import (
+ "context"
+
+ "github.com/opentracing/opentracing-go"
+)
+
+// Tags is a key-value map. It is used to set tags for a span
+type Tags map[string]any
+
+// StartSpan creates a new span with name and options (mostly tags). This function is a wrapper for
+// underlying tracing libraries. This method should only be used at the entrypoint of the program.
+func StartSpan(ctx context.Context, spanName string, tags Tags) (opentracing.Span, context.Context) {
+ return opentracing.StartSpanFromContext(ctx, spanName, tagsToOpentracingTags(tags)...)
+}
+
+// StartSpanIfHasParent creates a new span if the context already has an existing span. This function
+// adds a simple validation to prevent orphan spans outside interested code paths. It returns a dummy
+// span, which acts as normal span, but does absolutely nothing and is not recorded later.
+func StartSpanIfHasParent(ctx context.Context, spanName string, tags Tags) (opentracing.Span, context.Context) {
+ parent := opentracing.SpanFromContext(ctx)
+ if parent == nil {
+ return &NoopSpan{}, ctx
+ }
+ return opentracing.StartSpanFromContext(ctx, spanName, tagsToOpentracingTags(tags)...)
+}
+
+// DiscardSpanInContext discards the current active span from the context. This function is helpful
+// when the current code path enters an area shared by other code paths. Git catfile cache is a
+// good example of this type of span.
+func DiscardSpanInContext(ctx context.Context) context.Context {
+ if opentracing.SpanFromContext(ctx) == nil {
+ return ctx
+ }
+ return opentracing.ContextWithSpan(ctx, nil)
+}
+
+func tagsToOpentracingTags(tags Tags) []opentracing.StartSpanOption {
+ var opts []opentracing.StartSpanOption
+ for key, value := range tags {
+ opts = append(opts, opentracing.Tag{Key: key, Value: value})
+ }
+ return opts
+}
diff --git a/internal/tracing/tracing_test.go b/internal/tracing/tracing_test.go
new file mode 100644
index 000000000..191261f6f
--- /dev/null
+++ b/internal/tracing/tracing_test.go
@@ -0,0 +1,152 @@
+package tracing
+
+import (
+ "testing"
+
+ "github.com/opentracing/opentracing-go"
+ "github.com/opentracing/opentracing-go/log"
+ "github.com/stretchr/testify/assert"
+ "github.com/stretchr/testify/require"
+ "github.com/uber/jaeger-client-go"
+ "gitlab.com/gitlab-org/gitaly/v15/internal/testhelper"
+)
+
+func TestCreateSpan(t *testing.T) {
+ reporter := stubTracingReporter(t)
+ var span opentracing.Span
+ span, _ = StartSpan(testhelper.Context(t), "root", Tags{
+ "tagRoot1": "value1",
+ "tagRoot2": "value2",
+ "tagRoot3": "value3",
+ })
+ span.Finish()
+
+ require.Equal(t, []string{"root"}, reportedSpans(t, reporter))
+ require.Equal(t, Tags{
+ "tagRoot1": "value1",
+ "tagRoot2": "value2",
+ "tagRoot3": "value3",
+ }, spanTags(span))
+}
+
+func TestCreateSpanIfHasParent_emptyContext(t *testing.T) {
+ reporter := stubTracingReporter(t)
+ ctx := testhelper.Context(t)
+
+ var span, span2 opentracing.Span
+
+ span, ctx = StartSpanIfHasParent(ctx, "should-not-report-root", nil)
+ span.SetBaggageItem("baggage", "baggageValue")
+ span.SetTag("tag", "tagValue")
+ span.LogFields(log.String("log", "logValue"))
+ span.LogKV("log2", "logValue")
+ span.Finish()
+
+ span2, _ = StartSpanIfHasParent(ctx, "should-not-report-child", nil)
+ span2.Finish()
+
+ require.Empty(t, reportedSpans(t, reporter))
+}
+
+func TestCreateSpanIfHasParent_hasParent(t *testing.T) {
+ reporter := stubTracingReporter(t)
+ ctx := testhelper.Context(t)
+
+ var span1, span2 opentracing.Span
+ span1, ctx = StartSpan(ctx, "root", nil)
+ span2, _ = StartSpanIfHasParent(ctx, "child", nil)
+ span2.Finish()
+ span1.Finish()
+
+ spans := reportedSpans(t, reporter)
+ require.Equal(t, []string{"child", "root"}, spans)
+}
+
+func TestCreateSpanIfHasParent_hasParentWithTags(t *testing.T) {
+ reporter := stubTracingReporter(t)
+ ctx := testhelper.Context(t)
+
+ var span1, span2 opentracing.Span
+ span1, ctx = StartSpan(ctx, "root", Tags{
+ "tagRoot1": "value1",
+ "tagRoot2": "value2",
+ "tagRoot3": "value3",
+ })
+ span2, _ = StartSpanIfHasParent(ctx, "child", Tags{
+ "tagChild1": "value1",
+ "tagChild2": "value2",
+ "tagChild3": "value3",
+ })
+ span2.Finish()
+ span1.Finish()
+
+ spans := reportedSpans(t, reporter)
+ require.Equal(t, []string{"child", "root"}, spans)
+ require.Equal(t, Tags{
+ "tagRoot1": "value1",
+ "tagRoot2": "value2",
+ "tagRoot3": "value3",
+ }, spanTags(span1))
+ require.Equal(t, Tags{
+ "tagChild1": "value1",
+ "tagChild2": "value2",
+ "tagChild3": "value3",
+ }, spanTags(span2))
+}
+
+func TestDiscardSpanInContext_emptyContext(t *testing.T) {
+ ctx := DiscardSpanInContext(testhelper.Context(t))
+ require.Nil(t, opentracing.SpanFromContext(ctx))
+}
+
+func TestDiscardSpanInContext_hasParent(t *testing.T) {
+ reporter := stubTracingReporter(t)
+ ctx := testhelper.Context(t)
+
+ var span1, span2, span3 opentracing.Span
+ span1, ctx = StartSpan(ctx, "root", nil)
+ span2, ctx = StartSpanIfHasParent(ctx, "child", nil)
+ ctx = DiscardSpanInContext(ctx)
+ span3, _ = StartSpanIfHasParent(ctx, "discarded", nil)
+
+ span3.Finish()
+ span2.Finish()
+ span1.Finish()
+
+ spans := reportedSpans(t, reporter)
+ require.Equal(t, []string{"child", "root"}, spans)
+}
+
+func stubTracingReporter(t *testing.T) *jaeger.InMemoryReporter {
+ reporter := jaeger.NewInMemoryReporter()
+ tracer, tracerCloser := jaeger.NewTracer("", jaeger.NewConstSampler(true), reporter)
+ t.Cleanup(func() { testhelper.MustClose(t, tracerCloser) })
+
+ old := opentracing.GlobalTracer()
+ t.Cleanup(func() {
+ opentracing.SetGlobalTracer(old)
+ })
+ opentracing.SetGlobalTracer(tracer)
+ return reporter
+}
+
+func reportedSpans(t *testing.T, reporter *jaeger.InMemoryReporter) []string {
+ var names []string
+ for _, span := range reporter.GetSpans() {
+ if !assert.IsType(t, span, &jaeger.Span{}) {
+ continue
+ }
+ jaegerSpan := span.(*jaeger.Span)
+ names = append(names, jaegerSpan.OperationName())
+ }
+ return names
+}
+
+func spanTags(span opentracing.Span) Tags {
+ tags := Tags{}
+ jaegerSpan := span.(*jaeger.Span)
+ for key, value := range jaegerSpan.Tags() {
+ tags[key] = value
+ }
+ return tags
+}