From 003111bb5aebaed53fdf5059f5a60285eec6bdde Mon Sep 17 00:00:00 2001
From: Yuri Shkuro <yurishkuro@users.noreply.github.com>
Date: Sat, 30 Dec 2023 15:41:47 -0500
Subject: [PATCH 1/2] Remove grpc-gateway dependency (#5060)

## Which problem is this PR solving?
- Part of #5052
- Continues #5054
- Closes #4911

## Description of the changes
- Replace grpc-gateway based implementation with manual HTTP
implementation from #5054
- Clean up spurious grpc-gateway usage (e.g. all-in-one test that did
not need it)
- Delete grpc-gateway step from `make proto` and remove the
corresponding generated file
- `go mod tidy` removes grpc-gateway and github.com/golang/protobuf

## How was this change tested?
- CI

---------

Signed-off-by: Yuri Shkuro <github@ysh.us>
---
 Makefile.Protobuf.mk                     |   8 -
 cmd/all-in-one/all_in_one_test.go        |   4 +-
 cmd/query/app/apiv3/gateway_test.go      | 193 +++++++++++
 cmd/query/app/apiv3/grpc_gateway.go      |  73 -----
 cmd/query/app/apiv3/grpc_gateway_test.go | 392 -----------------------
 cmd/query/app/apiv3/http_gateway_test.go |  62 +++-
 cmd/query/app/server.go                  |  46 +--
 go.mod                                   |   4 +-
 go.sum                                   |   8 -
 proto-gen/api_v3/query_service.pb.gw.go  | 372 ---------------------
 10 files changed, 275 insertions(+), 887 deletions(-)
 create mode 100644 cmd/query/app/apiv3/gateway_test.go
 delete mode 100644 cmd/query/app/apiv3/grpc_gateway.go
 delete mode 100644 cmd/query/app/apiv3/grpc_gateway_test.go
 delete mode 100644 proto-gen/api_v3/query_service.pb.gw.go

diff --git a/Makefile.Protobuf.mk b/Makefile.Protobuf.mk
index 447c830e41f..141899672d7 100644
--- a/Makefile.Protobuf.mk
+++ b/Makefile.Protobuf.mk
@@ -77,7 +77,6 @@ define proto_compile
 
 endef
 
-# TODO add proto-hotrod to the list after regenerating its file (may need linter tweaking)
 .PHONY: proto
 proto: proto-model \
 	proto-api-v2 \
@@ -158,10 +157,3 @@ proto-api-v3:
 	find $(PATCHED_OTEL_PROTO_DIR) -name "*.proto" | xargs -L 1 $(SED) -i 's+go.opentelemetry.io/proto/otlp+github.com/jaegertracing/jaeger/proto-gen/otel+g'
 
 	$(call proto_compile, proto-gen/api_v3, idl/proto/api_v3/query_service.proto, -I$(PATCHED_OTEL_PROTO_DIR))
-
-	$(call print_caption, Generate API v3 gRPC Gateway)
-	$(PROTOC) \
-		$(PROTO_INCLUDES) \
-		-I$(PATCHED_OTEL_PROTO_DIR) \
- 		--grpc-gateway_out=logtostderr=true,grpc_api_configuration=idl/proto/api_v3/query_service_http.yaml,$(PROTO_GOGO_MAPPINGS):$(PWD)/proto-gen/api_v3 \
-		idl/proto/api_v3/query_service.proto
diff --git a/cmd/all-in-one/all_in_one_test.go b/cmd/all-in-one/all_in_one_test.go
index 4e6e8c6548a..70cca8d111b 100644
--- a/cmd/all-in-one/all_in_one_test.go
+++ b/cmd/all-in-one/all_in_one_test.go
@@ -26,7 +26,6 @@ import (
 	"time"
 
 	"github.com/gogo/protobuf/jsonpb"
-	"github.com/grpc-ecosystem/grpc-gateway/runtime"
 	"github.com/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
 
@@ -182,8 +181,7 @@ func getServicesAPIV3(t *testing.T) {
 	body, _ := io.ReadAll(resp.Body)
 
 	var servicesResponse api_v3.GetServicesResponse
-	jsonpb := runtime.JSONPb{}
-	err = jsonpb.Unmarshal(body, &servicesResponse)
+	err = json.Unmarshal(body, &servicesResponse)
 	require.NoError(t, err)
 	require.Len(t, servicesResponse.GetServices(), 1)
 	assert.Contains(t, servicesResponse.GetServices()[0], "jaeger")
diff --git a/cmd/query/app/apiv3/gateway_test.go b/cmd/query/app/apiv3/gateway_test.go
new file mode 100644
index 00000000000..34bc38709da
--- /dev/null
+++ b/cmd/query/app/apiv3/gateway_test.go
@@ -0,0 +1,193 @@
+// Copyright (c) 2021 The Jaeger Authors.
+//
+// Licensed 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 apiv3
+
+import (
+	"bytes"
+	"encoding/json"
+	"io"
+	"net/http"
+	"os"
+	"path"
+	"path/filepath"
+	"testing"
+
+	gogojsonpb "github.com/gogo/protobuf/jsonpb"
+	gogoproto "github.com/gogo/protobuf/proto"
+	"github.com/gorilla/mux"
+	"github.com/stretchr/testify/assert"
+	"github.com/stretchr/testify/require"
+
+	"github.com/jaegertracing/jaeger/model"
+	_ "github.com/jaegertracing/jaeger/pkg/gogocodec" // force gogo codec registration
+	"github.com/jaegertracing/jaeger/pkg/tenancy"
+	"github.com/jaegertracing/jaeger/proto-gen/api_v3"
+	"github.com/jaegertracing/jaeger/storage/spanstore"
+	spanstoremocks "github.com/jaegertracing/jaeger/storage/spanstore/mocks"
+)
+
+// Utility functions used from http_gateway_test.go.
+
+const (
+	snapshotLocation = "./snapshots/"
+)
+
+// Snapshots can be regenerated via:
+//
+//	REGENERATE_SNAPSHOTS=true go test -v ./cmd/query/app/apiv3/...
+var regenerateSnapshots = os.Getenv("REGENERATE_SNAPSHOTS") == "true"
+
+type testGateway struct {
+	reader *spanstoremocks.Reader
+	url    string
+	router *mux.Router
+	// used to set a tenancy header when executing requests
+	setupRequest func(*http.Request)
+}
+
+func (gw *testGateway) execRequest(t *testing.T, url string) ([]byte, int) {
+	req, err := http.NewRequest(http.MethodGet, gw.url+url, nil)
+	require.NoError(t, err)
+	req.Header.Set("Content-Type", "application/json")
+	gw.setupRequest(req)
+	response, err := http.DefaultClient.Do(req)
+	require.NoError(t, err)
+	body, err := io.ReadAll(response.Body)
+	require.NoError(t, err)
+	require.NoError(t, response.Body.Close())
+	return body, response.StatusCode
+}
+
+func (gw *testGateway) verifySnapshot(t *testing.T, body []byte) []byte {
+	// reformat JSON body with indentation, to make diffing easier
+	var data interface{}
+	require.NoError(t, json.Unmarshal(body, &data), "response: %s", string(body))
+	body, err := json.MarshalIndent(data, "", "  ")
+	require.NoError(t, err)
+
+	testName := path.Base(t.Name())
+	snapshotFile := filepath.Join(snapshotLocation, testName+".json")
+	if regenerateSnapshots {
+		os.WriteFile(snapshotFile, body, 0o644)
+	}
+	snapshot, err := os.ReadFile(snapshotFile)
+	require.NoError(t, err)
+	assert.Equal(t, string(snapshot), string(body), "comparing against stored snapshot. Use REGENERATE_SNAPSHOTS=true to rebuild snapshots.")
+	return body
+}
+
+func parseResponse(t *testing.T, body []byte, obj gogoproto.Message) {
+	require.NoError(t, gogojsonpb.Unmarshal(bytes.NewBuffer(body), obj))
+}
+
+func makeTestTrace() (*model.Trace, model.TraceID) {
+	traceID := model.NewTraceID(150, 160)
+	return &model.Trace{
+		Spans: []*model.Span{
+			{
+				TraceID:       traceID,
+				SpanID:        model.NewSpanID(180),
+				OperationName: "foobar",
+			},
+		},
+	}, traceID
+}
+
+func runGatewayTests(
+	t *testing.T,
+	basePath string,
+	tenancyOptions tenancy.Options,
+	setupRequest func(*http.Request),
+) {
+	gw := setupHTTPGateway(t, basePath, tenancyOptions)
+	gw.setupRequest = setupRequest
+	t.Run("GetServices", gw.runGatewayGetServices)
+	t.Run("GetOperations", gw.runGatewayGetOperations)
+	t.Run("GetTrace", gw.runGatewayGetTrace)
+	t.Run("FindTraces", gw.runGatewayFindTraces)
+}
+
+func (gw *testGateway) runGatewayGetServices(t *testing.T) {
+	gw.reader.On("GetServices", matchContext).Return([]string{"foo"}, nil).Once()
+
+	body, statusCode := gw.execRequest(t, "/api/v3/services")
+	require.Equal(t, http.StatusOK, statusCode)
+	body = gw.verifySnapshot(t, body)
+
+	var response api_v3.GetServicesResponse
+	parseResponse(t, body, &response)
+	assert.Equal(t, []string{"foo"}, response.Services)
+}
+
+func (gw *testGateway) runGatewayGetOperations(t *testing.T) {
+	qp := spanstore.OperationQueryParameters{ServiceName: "foo", SpanKind: "server"}
+	gw.reader.
+		On("GetOperations", matchContext, qp).
+		Return([]spanstore.Operation{{Name: "get_users", SpanKind: "server"}}, nil).Once()
+
+	body, statusCode := gw.execRequest(t, "/api/v3/operations?service=foo&span_kind=server")
+	require.Equal(t, http.StatusOK, statusCode)
+	body = gw.verifySnapshot(t, body)
+
+	var response api_v3.GetOperationsResponse
+	parseResponse(t, body, &response)
+	require.Len(t, response.Operations, 1)
+	assert.Equal(t, "get_users", response.Operations[0].Name)
+	assert.Equal(t, "server", response.Operations[0].SpanKind)
+}
+
+func (gw *testGateway) runGatewayGetTrace(t *testing.T) {
+	trace, traceID := makeTestTrace()
+	gw.reader.On("GetTrace", matchContext, traceID).Return(trace, nil).Once()
+
+	body, statusCode := gw.execRequest(t, "/api/v3/traces/"+traceID.String())
+	require.Equal(t, http.StatusOK, statusCode, "response=%s", string(body))
+	body = gw.verifySnapshot(t, body)
+
+	var response api_v3.GRPCGatewayWrapper
+	parseResponse(t, body, &response)
+
+	assert.Len(t, response.Result.ResourceSpans, 1)
+	assert.Equal(t,
+		bytesOfTraceID(t, traceID.High, traceID.Low),
+		response.Result.ResourceSpans[0].ScopeSpans[0].Spans[0].TraceId)
+}
+
+func (gw *testGateway) runGatewayFindTraces(t *testing.T) {
+	trace, traceID := makeTestTrace()
+	q, qp := mockFindQueries()
+	gw.reader.
+		On("FindTraces", matchContext, qp).
+		Return([]*model.Trace{trace}, nil).Once()
+	body, statusCode := gw.execRequest(t, "/api/v3/traces?"+q.Encode())
+	require.Equal(t, http.StatusOK, statusCode, "response=%s", string(body))
+	body = gw.verifySnapshot(t, body)
+
+	var response api_v3.GRPCGatewayWrapper
+	parseResponse(t, body, &response)
+
+	assert.Len(t, response.Result.ResourceSpans, 1)
+	assert.Equal(t,
+		bytesOfTraceID(t, traceID.High, traceID.Low),
+		response.Result.ResourceSpans[0].ScopeSpans[0].Spans[0].TraceId)
+}
+
+func bytesOfTraceID(t *testing.T, high, low uint64) []byte {
+	traceID := model.NewTraceID(high, low)
+	buf := make([]byte, 16)
+	_, err := traceID.MarshalTo(buf)
+	require.NoError(t, err)
+	return buf
+}
diff --git a/cmd/query/app/apiv3/grpc_gateway.go b/cmd/query/app/apiv3/grpc_gateway.go
deleted file mode 100644
index e93f024fa54..00000000000
--- a/cmd/query/app/apiv3/grpc_gateway.go
+++ /dev/null
@@ -1,73 +0,0 @@
-// Copyright (c) 2021 The Jaeger Authors.
-//
-// Licensed 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 apiv3
-
-import (
-	"context"
-	"net/http"
-
-	"github.com/gorilla/mux"
-	"github.com/grpc-ecosystem/grpc-gateway/runtime"
-	"go.uber.org/zap"
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/credentials"
-	"google.golang.org/grpc/credentials/insecure"
-
-	"github.com/jaegertracing/jaeger/pkg/config/tlscfg"
-	"github.com/jaegertracing/jaeger/pkg/netutils"
-	"github.com/jaegertracing/jaeger/pkg/tenancy"
-	"github.com/jaegertracing/jaeger/proto-gen/api_v3"
-)
-
-// RegisterGRPCGateway registers api_v3 endpoints into provided mux.
-func RegisterGRPCGateway(
-	ctx context.Context,
-	logger *zap.Logger,
-	r *mux.Router,
-	basePath string,
-	grpcEndpoint string,
-	grpcTLS *tlscfg.Options,
-	tm *tenancy.Manager,
-) error {
-	grpcEndpoint = netutils.FixLocalhost([]string{grpcEndpoint})[0]
-	jsonpb := &runtime.JSONPb{}
-
-	muxOpts := []runtime.ServeMuxOption{
-		runtime.WithMarshalerOption(runtime.MIMEWildcard, jsonpb),
-	}
-	if tm.Enabled {
-		muxOpts = append(muxOpts, runtime.WithMetadata(tm.MetadataAnnotator()))
-	}
-
-	grpcGatewayMux := runtime.NewServeMux(muxOpts...)
-	var handler http.Handler = grpcGatewayMux
-	if basePath != "/" {
-		handler = http.StripPrefix(basePath, grpcGatewayMux)
-	}
-	r.PathPrefix("/api/v3/").Handler(handler)
-
-	var dialOpts []grpc.DialOption
-	if grpcTLS.Enabled {
-		tlsCfg, err := grpcTLS.Config(logger)
-		if err != nil {
-			return err
-		}
-		creds := credentials.NewTLS(tlsCfg)
-		dialOpts = append(dialOpts, grpc.WithTransportCredentials(creds))
-	} else {
-		dialOpts = append(dialOpts, grpc.WithTransportCredentials(insecure.NewCredentials()))
-	}
-	return api_v3.RegisterQueryServiceHandlerFromEndpoint(ctx, grpcGatewayMux, grpcEndpoint, dialOpts)
-}
diff --git a/cmd/query/app/apiv3/grpc_gateway_test.go b/cmd/query/app/apiv3/grpc_gateway_test.go
deleted file mode 100644
index 381308b71ad..00000000000
--- a/cmd/query/app/apiv3/grpc_gateway_test.go
+++ /dev/null
@@ -1,392 +0,0 @@
-// Copyright (c) 2021 The Jaeger Authors.
-//
-// Licensed 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 apiv3
-
-import (
-	"bytes"
-	"context"
-	"encoding/json"
-	"fmt"
-	"io"
-	"net"
-	"net/http"
-	"os"
-	"path"
-	"path/filepath"
-	"strings"
-	"testing"
-
-	gogojsonpb "github.com/gogo/protobuf/jsonpb"
-	gogoproto "github.com/gogo/protobuf/proto"
-	"github.com/gorilla/mux"
-	"github.com/stretchr/testify/assert"
-	"github.com/stretchr/testify/require"
-	"go.uber.org/zap"
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/credentials"
-
-	"github.com/jaegertracing/jaeger/cmd/query/app/querysvc"
-	"github.com/jaegertracing/jaeger/model"
-	"github.com/jaegertracing/jaeger/pkg/config/tlscfg"
-	_ "github.com/jaegertracing/jaeger/pkg/gogocodec" // force gogo codec registration
-	"github.com/jaegertracing/jaeger/pkg/tenancy"
-	"github.com/jaegertracing/jaeger/proto-gen/api_v3"
-	dependencyStoreMocks "github.com/jaegertracing/jaeger/storage/dependencystore/mocks"
-	"github.com/jaegertracing/jaeger/storage/spanstore"
-	spanstoremocks "github.com/jaegertracing/jaeger/storage/spanstore/mocks"
-)
-
-const (
-	testCertKeyLocation = "../../../../pkg/config/tlscfg/testdata/"
-	snapshotLocation    = "./snapshots/"
-)
-
-// Snapshots can be regenerated via:
-//
-//	REGENERATE_SNAPSHOTS=true go test -v ./cmd/query/app/apiv3/...
-var regenerateSnapshots = os.Getenv("REGENERATE_SNAPSHOTS") == "true"
-
-// The tests in http_gateway_test.go set this to true to use manual gateway implementation.
-var useHTTPGateway = false
-
-type testGateway struct {
-	reader *spanstoremocks.Reader
-	url    string
-	router *mux.Router
-}
-
-type gatewayRequest struct {
-	url          string
-	setupRequest func(*http.Request)
-}
-
-func setupGRPCGateway(
-	t *testing.T,
-	basePath string,
-	serverTLS, clientTLS *tlscfg.Options,
-	tenancyOptions tenancy.Options,
-) *testGateway {
-	if useHTTPGateway {
-		return setupHTTPGateway(t, basePath, serverTLS, clientTLS, tenancyOptions)
-	}
-	gw := &testGateway{
-		reader: &spanstoremocks.Reader{},
-	}
-
-	q := querysvc.NewQueryService(gw.reader,
-		&dependencyStoreMocks.Reader{},
-		querysvc.QueryServiceOptions{},
-	)
-
-	var serverGRPCOpts []grpc.ServerOption
-	if serverTLS.Enabled {
-		config, err := serverTLS.Config(zap.NewNop())
-		require.NoError(t, err)
-		t.Cleanup(func() { serverTLS.Close() })
-		creds := credentials.NewTLS(config)
-		serverGRPCOpts = append(serverGRPCOpts, grpc.Creds(creds))
-	}
-	if tenancyOptions.Enabled {
-		tm := tenancy.NewManager(&tenancyOptions)
-		serverGRPCOpts = append(serverGRPCOpts,
-			grpc.StreamInterceptor(tenancy.NewGuardingStreamInterceptor(tm)),
-			grpc.UnaryInterceptor(tenancy.NewGuardingUnaryInterceptor(tm)),
-		)
-	}
-	grpcServer := grpc.NewServer(serverGRPCOpts...)
-	h := &Handler{
-		QueryService: q,
-	}
-	api_v3.RegisterQueryServiceServer(grpcServer, h)
-	lis, err := net.Listen("tcp", ":0")
-	require.NoError(t, err)
-
-	go func() {
-		err := grpcServer.Serve(lis)
-		require.NoError(t, err)
-	}()
-	t.Cleanup(func() { grpcServer.Stop() })
-
-	router := &mux.Router{}
-	router = router.PathPrefix(basePath).Subrouter()
-	ctx, cancel := context.WithCancel(context.Background())
-	err = RegisterGRPCGateway(
-		ctx, zap.NewNop(), router, basePath,
-		lis.Addr().String(), clientTLS, tenancy.NewManager(&tenancyOptions),
-	)
-	require.NoError(t, err)
-	t.Cleanup(func() { cancel() })
-	t.Cleanup(func() { clientTLS.Close() })
-
-	httpLis, err := net.Listen("tcp", ":0")
-	require.NoError(t, err)
-	httpServer := &http.Server{
-		Handler: router,
-	}
-	go func() {
-		err = httpServer.Serve(httpLis)
-		require.Equal(t, http.ErrServerClosed, err)
-	}()
-	t.Cleanup(func() { httpServer.Shutdown(context.Background()) })
-
-	gw.url = fmt.Sprintf(
-		"http://localhost%s%s",
-		strings.Replace(httpLis.Addr().String(), "[::]", "", 1),
-		basePath)
-	return gw
-}
-
-func (gw *testGateway) execRequest(t *testing.T, gwReq *gatewayRequest) ([]byte, int) {
-	req, err := http.NewRequest(http.MethodGet, gw.url+gwReq.url, nil)
-	require.NoError(t, err)
-	req.Header.Set("Content-Type", "application/json")
-	gwReq.setupRequest(req)
-	response, err := http.DefaultClient.Do(req)
-	require.NoError(t, err)
-	body, err := io.ReadAll(response.Body)
-	require.NoError(t, err)
-	require.NoError(t, response.Body.Close())
-	return body, response.StatusCode
-}
-
-func verifySnapshot(t *testing.T, body []byte) []byte {
-	// reformat JSON body with indentation, to make diffing easier
-	var data interface{}
-	require.NoError(t, json.Unmarshal(body, &data), "response: %s", string(body))
-	body, err := json.MarshalIndent(data, "", "  ")
-	require.NoError(t, err)
-
-	testName := path.Base(t.Name())
-	snapshotFile := filepath.Join(snapshotLocation, testName+".json")
-	if regenerateSnapshots {
-		os.WriteFile(snapshotFile, body, 0o644)
-	}
-	snapshot, err := os.ReadFile(snapshotFile)
-	require.NoError(t, err)
-	assert.Equal(t, string(snapshot), string(body), "comparing against stored snapshot. Use REGENERATE_SNAPSHOTS=true to rebuild snapshots.")
-	return body
-}
-
-func parseResponse(t *testing.T, body []byte, obj gogoproto.Message) {
-	require.NoError(t, gogojsonpb.Unmarshal(bytes.NewBuffer(body), obj))
-}
-
-func makeTestTrace() (*model.Trace, model.TraceID) {
-	traceID := model.NewTraceID(150, 160)
-	return &model.Trace{
-		Spans: []*model.Span{
-			{
-				TraceID:       traceID,
-				SpanID:        model.NewSpanID(180),
-				OperationName: "foobar",
-			},
-		},
-	}, traceID
-}
-
-func testGRPCGateway(
-	t *testing.T, basePath string,
-	serverTLS, clientTLS *tlscfg.Options,
-) {
-	testGRPCGatewayWithTenancy(t, basePath, serverTLS, clientTLS,
-		tenancy.Options{
-			Enabled: false,
-		},
-		func(*http.Request) { /* setupRequest : no changes */ },
-	)
-}
-
-func testGRPCGatewayWithTenancy(
-	t *testing.T,
-	basePath string,
-	serverTLS *tlscfg.Options,
-	clientTLS *tlscfg.Options,
-	tenancyOptions tenancy.Options,
-	setupRequest func(*http.Request),
-) {
-	gw := setupGRPCGateway(t, basePath, serverTLS, clientTLS, tenancyOptions)
-	t.Run("GetServices", func(t *testing.T) {
-		runGatewayGetServices(t, gw, setupRequest)
-	})
-	t.Run("GetOperations", func(t *testing.T) {
-		runGatewayGetOperations(t, gw, setupRequest)
-	})
-	t.Run("GetTrace", func(t *testing.T) {
-		runGatewayGetTrace(t, gw, setupRequest)
-	})
-	t.Run("FindTraces", func(t *testing.T) {
-		runGatewayFindTraces(t, gw, setupRequest)
-	})
-}
-
-func runGatewayGetServices(t *testing.T, gw *testGateway, setupRequest func(*http.Request)) {
-	gw.reader.On("GetServices", matchContext).Return([]string{"foo"}, nil).Once()
-
-	body, statusCode := gw.execRequest(t, &gatewayRequest{
-		url:          "/api/v3/services",
-		setupRequest: setupRequest,
-	})
-	require.Equal(t, http.StatusOK, statusCode)
-	body = verifySnapshot(t, body)
-
-	var response api_v3.GetServicesResponse
-	parseResponse(t, body, &response)
-	assert.Equal(t, []string{"foo"}, response.Services)
-}
-
-func runGatewayGetOperations(t *testing.T, gw *testGateway, setupRequest func(*http.Request)) {
-	qp := spanstore.OperationQueryParameters{ServiceName: "foo", SpanKind: "server"}
-	gw.reader.
-		On("GetOperations", matchContext, qp).
-		Return([]spanstore.Operation{{Name: "get_users", SpanKind: "server"}}, nil).Once()
-
-	body, statusCode := gw.execRequest(t, &gatewayRequest{
-		url:          "/api/v3/operations?service=foo&span_kind=server",
-		setupRequest: setupRequest,
-	})
-	require.Equal(t, http.StatusOK, statusCode)
-	body = verifySnapshot(t, body)
-
-	var response api_v3.GetOperationsResponse
-	parseResponse(t, body, &response)
-	require.Len(t, response.Operations, 1)
-	assert.Equal(t, "get_users", response.Operations[0].Name)
-	assert.Equal(t, "server", response.Operations[0].SpanKind)
-}
-
-func runGatewayGetTrace(t *testing.T, gw *testGateway, setupRequest func(*http.Request)) {
-	trace, traceID := makeTestTrace()
-	gw.reader.On("GetTrace", matchContext, traceID).Return(trace, nil).Once()
-
-	body, statusCode := gw.execRequest(t, &gatewayRequest{
-		url:          "/api/v3/traces/" + traceID.String(), // hex string
-		setupRequest: setupRequest,
-	})
-	require.Equal(t, http.StatusOK, statusCode, "response=%s", string(body))
-	body = verifySnapshot(t, body)
-
-	var response api_v3.GRPCGatewayWrapper
-	parseResponse(t, body, &response)
-
-	assert.Len(t, response.Result.ResourceSpans, 1)
-	assert.Equal(t,
-		bytesOfTraceID(t, traceID.High, traceID.Low),
-		response.Result.ResourceSpans[0].ScopeSpans[0].Spans[0].TraceId)
-}
-
-func runGatewayFindTraces(t *testing.T, gw *testGateway, setupRequest func(*http.Request)) {
-	trace, traceID := makeTestTrace()
-	q, qp := mockFindQueries()
-	gw.reader.
-		On("FindTraces", matchContext, qp).
-		Return([]*model.Trace{trace}, nil).Once()
-	body, statusCode := gw.execRequest(t, &gatewayRequest{
-		url:          "/api/v3/traces?" + q.Encode(),
-		setupRequest: setupRequest,
-	})
-	require.Equal(t, http.StatusOK, statusCode, "response=%s", string(body))
-	body = verifySnapshot(t, body)
-
-	var response api_v3.GRPCGatewayWrapper
-	parseResponse(t, body, &response)
-
-	assert.Len(t, response.Result.ResourceSpans, 1)
-	assert.Equal(t,
-		bytesOfTraceID(t, traceID.High, traceID.Low),
-		response.Result.ResourceSpans[0].ScopeSpans[0].Spans[0].TraceId)
-}
-
-func bytesOfTraceID(t *testing.T, high, low uint64) []byte {
-	traceID := model.NewTraceID(high, low)
-	buf := make([]byte, 16)
-	_, err := traceID.MarshalTo(buf)
-	require.NoError(t, err)
-	return buf
-}
-
-func TestGRPCGateway(t *testing.T) {
-	testGRPCGateway(t, "/", &tlscfg.Options{}, &tlscfg.Options{})
-}
-
-func TestGRPCGatewayWithBasePathAndTLS(t *testing.T) {
-	serverTLS := &tlscfg.Options{
-		Enabled:  true,
-		CAPath:   testCertKeyLocation + "/example-CA-cert.pem",
-		CertPath: testCertKeyLocation + "/example-server-cert.pem",
-		KeyPath:  testCertKeyLocation + "/example-server-key.pem",
-	}
-	clientTLS := &tlscfg.Options{
-		Enabled:    true,
-		CAPath:     testCertKeyLocation + "/example-CA-cert.pem",
-		CertPath:   testCertKeyLocation + "/example-client-cert.pem",
-		KeyPath:    testCertKeyLocation + "/example-client-key.pem",
-		ServerName: "example.com",
-	}
-	testGRPCGateway(t, "/jaeger", serverTLS, clientTLS)
-}
-
-func TestGRPCGatewayWithTenancy(t *testing.T) {
-	tenancyOptions := tenancy.Options{
-		Enabled: true,
-	}
-	tm := tenancy.NewManager(&tenancyOptions)
-	testGRPCGatewayWithTenancy(t, "/", &tlscfg.Options{}, &tlscfg.Options{},
-		// Configure the gateway to forward tenancy header from HTTP to GRPC
-		tenancyOptions,
-		// Add a tenancy header on outbound requests
-		func(req *http.Request) {
-			req.Header.Add(tm.Header, "dummy")
-		})
-}
-
-func TestGRPCGatewayTenancyRejection(t *testing.T) {
-	basePath := "/"
-	tenancyOptions := tenancy.Options{Enabled: true}
-	gw := setupGRPCGateway(t,
-		basePath, &tlscfg.Options{}, &tlscfg.Options{},
-		tenancyOptions)
-
-	traceID := model.NewTraceID(150, 160)
-	gw.reader.On("GetTrace", matchContext, matchTraceID).Return(
-		&model.Trace{
-			Spans: []*model.Span{
-				{
-					TraceID:       traceID,
-					SpanID:        model.NewSpanID(180),
-					OperationName: "foobar",
-				},
-			},
-		}, nil).Once()
-
-	req, err := http.NewRequest(http.MethodGet, gw.url+"/api/v3/traces/123", nil)
-	require.NoError(t, err)
-	req.Header.Set("Content-Type", "application/json")
-	// We don't set tenant header
-	response, err := http.DefaultClient.Do(req)
-	require.NoError(t, err)
-	body, err := io.ReadAll(response.Body)
-	require.NoError(t, err)
-	require.NoError(t, response.Body.Close())
-	require.Equal(t, http.StatusUnauthorized, response.StatusCode, "response=%s", string(body))
-
-	// Try again with tenant header set
-	tm := tenancy.NewManager(&tenancyOptions)
-	req.Header.Set(tm.Header, "acme")
-	response, err = http.DefaultClient.Do(req)
-	require.NoError(t, err)
-	require.NoError(t, response.Body.Close())
-	require.Equal(t, http.StatusOK, response.StatusCode)
-	// Skip unmarshal of response; it is enough that it succeeded
-}
diff --git a/cmd/query/app/apiv3/http_gateway_test.go b/cmd/query/app/apiv3/http_gateway_test.go
index 86300ff5b50..dc0a7f7f35c 100644
--- a/cmd/query/app/apiv3/http_gateway_test.go
+++ b/cmd/query/app/apiv3/http_gateway_test.go
@@ -5,6 +5,7 @@ package apiv3
 
 import (
 	"fmt"
+	"io"
 	"net/http"
 	"net/http/httptest"
 	"net/url"
@@ -18,7 +19,6 @@ import (
 
 	"github.com/jaegertracing/jaeger/cmd/query/app/querysvc"
 	"github.com/jaegertracing/jaeger/model"
-	"github.com/jaegertracing/jaeger/pkg/config/tlscfg"
 	"github.com/jaegertracing/jaeger/pkg/jtracer"
 	"github.com/jaegertracing/jaeger/pkg/tenancy"
 	"github.com/jaegertracing/jaeger/pkg/testutils"
@@ -60,7 +60,6 @@ func setupHTTPGatewayNoServer(
 func setupHTTPGateway(
 	t *testing.T,
 	basePath string,
-	serverTLS, clientTLS *tlscfg.Options,
 	tenancyOptions tenancy.Options,
 ) *testGateway {
 	gw := setupHTTPGatewayNoServer(t, basePath, tenancyOptions)
@@ -76,11 +75,22 @@ func setupHTTPGateway(
 }
 
 func TestHTTPGateway(t *testing.T) {
-	useHTTPGateway = true
-	t.Cleanup(func() { useHTTPGateway = false })
-	t.Run("TestGRPCGateway", TestGRPCGateway)
-	t.Run("TestGRPCGatewayWithTenancy", TestGRPCGatewayWithTenancy)
-	t.Run("TestGRPCGatewayTenancyRejection", TestGRPCGatewayTenancyRejection)
+	for _, ten := range []bool{false, true} {
+		t.Run(fmt.Sprintf("tenancy=%v", ten), func(t *testing.T) {
+			tenancyOptions := tenancy.Options{
+				Enabled: ten,
+			}
+			tm := tenancy.NewManager(&tenancyOptions)
+			runGatewayTests(t, "/",
+				tenancyOptions,
+				func(req *http.Request) {
+					if ten {
+						// Add a tenancy header on outbound requests
+						req.Header.Add(tm.Header, "dummy")
+					}
+				})
+		})
+	}
 }
 
 func TestHTTPGatewayTryHandleError(t *testing.T) {
@@ -274,3 +284,41 @@ func TestHTTPGatewayGetOperationsErrors(t *testing.T) {
 	gw.router.ServeHTTP(w, r)
 	assert.Contains(t, w.Body.String(), simErr)
 }
+
+func TestHTTPGatewayTenancyRejection(t *testing.T) {
+	basePath := "/"
+	tenancyOptions := tenancy.Options{Enabled: true}
+	gw := setupHTTPGateway(t, basePath, tenancyOptions)
+
+	traceID := model.NewTraceID(150, 160)
+	gw.reader.On("GetTrace", matchContext, matchTraceID).Return(
+		&model.Trace{
+			Spans: []*model.Span{
+				{
+					TraceID:       traceID,
+					SpanID:        model.NewSpanID(180),
+					OperationName: "foobar",
+				},
+			},
+		}, nil).Once()
+
+	req, err := http.NewRequest(http.MethodGet, gw.url+"/api/v3/traces/123", nil)
+	require.NoError(t, err)
+	req.Header.Set("Content-Type", "application/json")
+	// We don't set tenant header
+	response, err := http.DefaultClient.Do(req)
+	require.NoError(t, err)
+	body, err := io.ReadAll(response.Body)
+	require.NoError(t, err)
+	require.NoError(t, response.Body.Close())
+	require.Equal(t, http.StatusUnauthorized, response.StatusCode, "response=%s", string(body))
+
+	// Try again with tenant header set
+	tm := tenancy.NewManager(&tenancyOptions)
+	req.Header.Set(tm.Header, "acme")
+	response, err = http.DefaultClient.Do(req)
+	require.NoError(t, err)
+	require.NoError(t, response.Body.Close())
+	require.Equal(t, http.StatusOK, response.StatusCode)
+	// Skip unmarshal of response; it is enough that it succeeded
+}
diff --git a/cmd/query/app/server.go b/cmd/query/app/server.go
index 662ad4a6834..27913516fee 100644
--- a/cmd/query/app/server.go
+++ b/cmd/query/app/server.go
@@ -15,7 +15,6 @@
 package app
 
 import (
-	"context"
 	"errors"
 	"fmt"
 	"net"
@@ -62,7 +61,6 @@ type Server struct {
 	httpServer         *http.Server
 	separatePorts      bool
 	unavailableChannel chan healthcheck.Status
-	grpcGatewayCancel  context.CancelFunc
 }
 
 // NewServer creates and initializes Server
@@ -85,7 +83,7 @@ func NewServer(logger *zap.Logger, querySvc *querysvc.QueryService, metricsQuery
 		return nil, err
 	}
 
-	httpServer, closeGRPCGateway, err := createHTTPServer(querySvc, metricsQuerySvc, options, tm, tracer, logger)
+	httpServer, err := createHTTPServer(querySvc, metricsQuerySvc, options, tm, tracer, logger)
 	if err != nil {
 		return nil, err
 	}
@@ -99,7 +97,6 @@ func NewServer(logger *zap.Logger, querySvc *querysvc.QueryService, metricsQuery
 		httpServer:         httpServer,
 		separatePorts:      grpcPort != httpPort,
 		unavailableChannel: make(chan healthcheck.Status),
-		grpcGatewayCancel:  closeGRPCGateway,
 	}, nil
 }
 
@@ -149,7 +146,14 @@ func createGRPCServer(querySvc *querysvc.QueryService, metricsQuerySvc querysvc.
 	return server, nil
 }
 
-func createHTTPServer(querySvc *querysvc.QueryService, metricsQuerySvc querysvc.MetricsQueryService, queryOpts *QueryOptions, tm *tenancy.Manager, tracer *jtracer.JTracer, logger *zap.Logger) (*http.Server, context.CancelFunc, error) {
+func createHTTPServer(
+	querySvc *querysvc.QueryService,
+	metricsQuerySvc querysvc.MetricsQueryService,
+	queryOpts *QueryOptions,
+	tm *tenancy.Manager,
+	tracer *jtracer.JTracer,
+	logger *zap.Logger,
+) (*http.Server, error) {
 	apiHandlerOptions := []HandlerOption{
 		HandlerOptions.Logger(logger),
 		HandlerOptions.Tracer(tracer),
@@ -165,11 +169,12 @@ func createHTTPServer(querySvc *querysvc.QueryService, metricsQuerySvc querysvc.
 		r = r.PathPrefix(queryOpts.BasePath).Subrouter()
 	}
 
-	ctx, closeGRPCGateway := context.WithCancel(context.Background())
-	if err := apiv3.RegisterGRPCGateway(ctx, logger, r, queryOpts.BasePath, queryOpts.GRPCHostPort, &queryOpts.TLSGRPC, tm); err != nil {
-		closeGRPCGateway()
-		return nil, nil, err
-	}
+	(&apiv3.HTTPGateway{
+		QueryService: querySvc,
+		TenancyMgr:   tm,
+		Logger:       logger,
+		Tracer:       tracer,
+	}).RegisterRoutes(r)
 
 	apiHandler.RegisterRoutes(r)
 	RegisterStaticHandler(r, logger, queryOpts, querySvc.GetCapabilities())
@@ -191,13 +196,12 @@ func createHTTPServer(querySvc *querysvc.QueryService, metricsQuerySvc querysvc.
 	if queryOpts.TLSHTTP.Enabled {
 		tlsCfg, err := queryOpts.TLSHTTP.Config(logger) // This checks if the certificates are correctly provided
 		if err != nil {
-			closeGRPCGateway()
-			return nil, nil, err
+			return nil, err
 		}
 		server.TLSConfig = tlsCfg
 
 	}
-	return server, closeGRPCGateway, nil
+	return server, nil
 }
 
 // initListener initialises listeners of the server
@@ -320,17 +324,17 @@ func (s *Server) Start() error {
 
 // Close stops http, GRPC servers and closes the port listener.
 func (s *Server) Close() error {
-	s.grpcGatewayCancel()
-	s.queryOptions.TLSGRPC.Close()
-	s.queryOptions.TLSHTTP.Close()
+	var errs []error
+	errs = append(errs, s.queryOptions.TLSGRPC.Close())
+	errs = append(errs, s.queryOptions.TLSHTTP.Close())
 	s.grpcServer.Stop()
-	s.httpServer.Close()
+	errs = append(errs, s.httpServer.Close())
 	if s.separatePorts {
-		s.httpConn.Close()
-		s.grpcConn.Close()
+		errs = append(errs, s.httpConn.Close())
+		errs = append(errs, s.grpcConn.Close())
 	} else {
 		s.cmuxServer.Close()
-		s.conn.Close()
+		errs = append(errs, s.conn.Close())
 	}
-	return nil
+	return errors.Join(errs...)
 }
diff --git a/go.mod b/go.mod
index 9f727881a99..74db3fca9b3 100644
--- a/go.mod
+++ b/go.mod
@@ -17,11 +17,9 @@ require (
 	github.com/gocql/gocql v1.3.2
 	github.com/gogo/googleapis v1.4.1
 	github.com/gogo/protobuf v1.3.2
-	github.com/golang/protobuf v1.5.3
 	github.com/gorilla/handlers v1.5.1
 	github.com/gorilla/mux v1.8.1
 	github.com/grpc-ecosystem/go-grpc-middleware v1.4.0
-	github.com/grpc-ecosystem/grpc-gateway v1.16.0
 	github.com/hashicorp/go-hclog v1.6.1
 	github.com/hashicorp/go-plugin v1.6.0
 	github.com/kr/pretty v0.3.1
@@ -109,6 +107,7 @@ require (
 	github.com/go-ole/go-ole v1.2.6 // indirect
 	github.com/golang/glog v1.1.2 // indirect
 	github.com/golang/groupcache v0.0.0-20210331224755-41bb18bfe9da // indirect
+	github.com/golang/protobuf v1.5.3 // indirect
 	github.com/golang/snappy v0.0.4 // indirect
 	github.com/google/flatbuffers v1.12.1 // indirect
 	github.com/google/uuid v1.4.0 // indirect
@@ -216,7 +215,6 @@ require (
 	golang.org/x/exp v0.0.0-20231127185646-65229373498e // indirect
 	golang.org/x/text v0.14.0 // indirect
 	gonum.org/v1/gonum v0.14.0 // indirect
-	google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f // indirect
 	google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 // indirect
 	google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 // indirect
 	gopkg.in/inf.v0 v0.9.1 // indirect
diff --git a/go.sum b/go.sum
index ee54e8ce343..c60719cc071 100644
--- a/go.sum
+++ b/go.sum
@@ -57,7 +57,6 @@ github.com/alecthomas/units v0.0.0-20151022065526-2efee857e7cf/go.mod h1:ybxpYRF
 github.com/alecthomas/units v0.0.0-20190717042225-c3de453c63f4/go.mod h1:ybxpYRFXyAe+OPACYpWeL0wqObRcbAqCMya13uyzqw0=
 github.com/alecthomas/units v0.0.0-20190924025748-f65c72e2690d/go.mod h1:rBZYJk541a8SKzHPHnH3zbiI+7dagKZ0cgpgrD7Fyho=
 github.com/alecthomas/units v0.0.0-20211218093645-b94a6e3cc137/go.mod h1:OMCwj8VM1Kc9e19TLln2VL61YJF0x1XFtfdL4JdbSyE=
-github.com/antihax/optional v1.0.0/go.mod h1:uupD/76wgC+ih3iEmQUL+0Ugr19nfwCT1kdvxnR2qWY=
 github.com/apache/thrift v0.19.0 h1:sOqkWPzMj7w6XaYbJQG7m4sGqVolaW/0D28Ln7yPzMk=
 github.com/apache/thrift v0.19.0/go.mod h1:SUALL216IiaOw2Oy+5Vs9lboJ/t9g40C+G07Dc0QC1I=
 github.com/armon/consul-api v0.0.0-20180202201655-eb2c6b5be1b6/go.mod h1:grANhF5doyWs3UAsr3K4I6qtAmlQcZDesFNEHPZAzj8=
@@ -146,7 +145,6 @@ github.com/fsnotify/fsnotify v1.4.7/go.mod h1:jwhsz4b93w/PPRr/qN1Yymfu8t87LnFCMo
 github.com/fsnotify/fsnotify v1.4.9/go.mod h1:znqG4EE+3YCdAaPaxE2ZRY/06pZUdp0tY4IgpuI1SZQ=
 github.com/fsnotify/fsnotify v1.7.0 h1:8JEhPFa5W2WU7YfeZzPNqzMP6Lwt7L2715Ggo0nosvA=
 github.com/fsnotify/fsnotify v1.7.0/go.mod h1:40Bi/Hjc2AVfZrqy+aj+yEI+/bRxZnMJyTJwOpGvigM=
-github.com/ghodss/yaml v1.0.0/go.mod h1:4dBDuWmgqj2HViK6kFavaiC9ZROes6MMH2rRYeMEF04=
 github.com/go-gl/glfw v0.0.0-20190409004039-e6da0acd62b1/go.mod h1:vR7hzQXu2zJy9AVAgeJqvqgH9Q5CA+iKCZ2gyEVpxRU=
 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20191125211704-12ad95a8df72/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
 github.com/go-gl/glfw/v3.3/glfw v0.0.0-20200222043503-6f7a984d4dc4/go.mod h1:tQ2UAYgL5IevRw8kRxooKSPJfGvJ9fJQFa0TUsXzTg8=
@@ -263,8 +261,6 @@ github.com/gorilla/securecookie v1.1.1/go.mod h1:ra0sb63/xPlUeL+yeDciTfxMRAA+MP+
 github.com/gorilla/sessions v1.2.1/go.mod h1:dk2InVEVJ0sfLlnXv9EAgkf6ecYs/i80K/zI+bUmuGM=
 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0 h1:UH//fgunKIs4JdUbpDl1VZCDaL56wXCB/5+wF6uHfaI=
 github.com/grpc-ecosystem/go-grpc-middleware v1.4.0/go.mod h1:g5qyo/la0ALbONm6Vbp88Yd8NsDy6rZz+RcrMPxvld8=
-github.com/grpc-ecosystem/grpc-gateway v1.16.0 h1:gmcG1KaJ57LophUzW0Hy8NmPhnMZb4M0+kPpLofRdBo=
-github.com/grpc-ecosystem/grpc-gateway v1.16.0/go.mod h1:BDjrQk3hbvj6Nolgz8mAMFbcEtjT1g+wF4CSlocrBnw=
 github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0 h1:YBftPWNWd4WwGqtY2yeZL2ef8rHAxPBD8KFhJpmcqms=
 github.com/grpc-ecosystem/grpc-gateway/v2 v2.16.0/go.mod h1:YN5jB8ie0yfIUg6VvR9Kz84aCaG7AsGZnLjhHbUqwPg=
 github.com/hailocab/go-hostpool v0.0.0-20160125115350-e80d13ce29ed h1:5upAirOpQc1Q53c0bnx2ufif5kANL7bfZWcc6VJWJd8=
@@ -495,7 +491,6 @@ github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475 h1:N/ElC8H3+5X
 github.com/rcrowley/go-metrics v0.0.0-20201227073835-cf1acfcdf475/go.mod h1:bCqnVzQkZxMG4s8nGwiZ5l3QUCyqpo9Y+/ZMZ9VjZe4=
 github.com/relvacode/iso8601 v1.3.0 h1:HguUjsGpIMh/zsTczGN3DVJFxTU/GX+MMmzcKoMO7ko=
 github.com/relvacode/iso8601 v1.3.0/go.mod h1:FlNp+jz+TXpyRqgmM7tnzHHzBnz776kmAH2h3sZCn0I=
-github.com/rogpeppe/fastuuid v1.2.0/go.mod h1:jVj6XXZzXRy/MSR5jhDC/2q6DgLz+nrA6LYCDYWNEvQ=
 github.com/rogpeppe/go-internal v1.3.0/go.mod h1:M8bDsm7K2OlrFYOpmOWEs/qY81heoFRclV5y23lUDJ4=
 github.com/rogpeppe/go-internal v1.6.1/go.mod h1:xXDCJY+GAPziupqXw64V24skbSoqbTEfhy4qGm1nDQc=
 github.com/rogpeppe/go-internal v1.9.0/go.mod h1:WtVeX8xhTBvf0smdhujwtBcq4Qrzq/fJaraNFVN+nFs=
@@ -1020,7 +1015,6 @@ google.golang.org/genproto v0.0.0-20200331122359-1ee6d9798940/go.mod h1:55QSHmfG
 google.golang.org/genproto v0.0.0-20200423170343-7949de9c1215/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
 google.golang.org/genproto v0.0.0-20200430143042-b979b6f78d84/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
 google.golang.org/genproto v0.0.0-20200511104702-f5ebc3bea380/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
-google.golang.org/genproto v0.0.0-20200513103714-09dca8ec2884/go.mod h1:55QSHmfGQM9UVYDPBsyGGes0y52j32PQ3BqQfXhyH3c=
 google.golang.org/genproto v0.0.0-20200515170657-fc4c6c6a6587/go.mod h1:YsZOwe1myG/8QRHRsmBRE1LrgQY60beZKjly0O1fX9U=
 google.golang.org/genproto v0.0.0-20200526211855-cb27e3aa2013/go.mod h1:NbSheEEYHJ7i3ixzK3sjbqSGDJWnxyFXZblF3eUsNvo=
 google.golang.org/genproto v0.0.0-20200618031413-b414f8b61790/go.mod h1:jDfRM7FcilCzHH/e9qn6dsT145K34l5v+OpcnNgKAAA=
@@ -1028,7 +1022,6 @@ google.golang.org/genproto v0.0.0-20200729003335-053ba62fc06f/go.mod h1:FWY/as6D
 google.golang.org/genproto v0.0.0-20200804131852-c06518451d9c/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
 google.golang.org/genproto v0.0.0-20200825200019-8632dd797987/go.mod h1:FWY/as6DDZQgahTzZj3fqbO1CbirC29ZNUFHwi0/+no=
 google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f h1:Vn+VyHU5guc9KjB5KrjI2q0wCOWEOIh0OEsleqakHJg=
-google.golang.org/genproto v0.0.0-20231120223509-83a465c0220f/go.mod h1:nWSwAFPb+qfNJXsoeO3Io7zf4tMSfN8EA8RlDA04GhY=
 google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17 h1:JpwMPBpFN3uKhdaekDpiNlImDdkUAyiJ6ez/uxGaUSo=
 google.golang.org/genproto/googleapis/api v0.0.0-20231106174013-bbf56f31fb17/go.mod h1:0xJLfVdJqpAPl8tDg1ujOCGzx6LFLttXT5NhllGOXY4=
 google.golang.org/genproto/googleapis/rpc v0.0.0-20231127180814-3a041ad873d4 h1:DC7wcm+i+P1rN3Ff07vL+OndGg5OhNddHyTA+ocPqYE=
@@ -1045,7 +1038,6 @@ google.golang.org/grpc v1.28.0/go.mod h1:rpkK4SK4GF4Ach/+MFLZUBavHOvF2JJB5uozKKa
 google.golang.org/grpc v1.29.1/go.mod h1:itym6AZVZYACWQqET3MqgPpjcuV5QH3BxFS3IjizoKk=
 google.golang.org/grpc v1.30.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak=
 google.golang.org/grpc v1.31.0/go.mod h1:N36X2cJ7JwdamYAgDz+s+rVMFjt3numwzf/HckM8pak=
-google.golang.org/grpc v1.33.1/go.mod h1:fr5YgcSWrqhRRxogOsw7RzIpsmvOZ6IcH4kBYTpR3n0=
 google.golang.org/grpc v1.33.2/go.mod h1:JMHMWHQWaTccqQQlmk3MJZS+GWXOdAesneDmEnv2fbc=
 google.golang.org/grpc v1.60.0 h1:6FQAR0kM31P6MRdeluor2w2gPaS4SVNrD/DNTxrQ15k=
 google.golang.org/grpc v1.60.0/go.mod h1:OlCHIeLYqSSsLi6i49B5QGdzaMZK9+M7LXN2FKz4eGM=
diff --git a/proto-gen/api_v3/query_service.pb.gw.go b/proto-gen/api_v3/query_service.pb.gw.go
deleted file mode 100644
index d393094c306..00000000000
--- a/proto-gen/api_v3/query_service.pb.gw.go
+++ /dev/null
@@ -1,372 +0,0 @@
-// Code generated by protoc-gen-grpc-gateway. DO NOT EDIT.
-// source: query_service.proto
-
-/*
-Package api_v3 is a reverse proxy.
-
-It translates gRPC into RESTful JSON APIs.
-*/
-package api_v3
-
-import (
-	"context"
-	"io"
-	"net/http"
-
-	"github.com/golang/protobuf/descriptor"
-	"github.com/golang/protobuf/proto"
-	"github.com/grpc-ecosystem/grpc-gateway/runtime"
-	"github.com/grpc-ecosystem/grpc-gateway/utilities"
-	"google.golang.org/grpc"
-	"google.golang.org/grpc/codes"
-	"google.golang.org/grpc/grpclog"
-	"google.golang.org/grpc/metadata"
-	"google.golang.org/grpc/status"
-)
-
-// Suppress "imported and not used" errors
-var _ codes.Code
-var _ io.Reader
-var _ status.Status
-var _ = runtime.String
-var _ = utilities.NewDoubleArray
-var _ = descriptor.ForMessage
-var _ = metadata.Join
-
-var (
-	filter_QueryService_GetTrace_0 = &utilities.DoubleArray{Encoding: map[string]int{"trace_id": 0}, Base: []int{1, 1, 0}, Check: []int{0, 1, 2}}
-)
-
-func request_QueryService_GetTrace_0(ctx context.Context, marshaler runtime.Marshaler, client QueryServiceClient, req *http.Request, pathParams map[string]string) (QueryService_GetTraceClient, runtime.ServerMetadata, error) {
-	var protoReq GetTraceRequest
-	var metadata runtime.ServerMetadata
-
-	var (
-		val string
-		ok  bool
-		err error
-		_   = err
-	)
-
-	val, ok = pathParams["trace_id"]
-	if !ok {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "missing parameter %s", "trace_id")
-	}
-
-	protoReq.TraceId, err = runtime.String(val)
-
-	if err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "type mismatch, parameter: %s, error: %v", "trace_id", err)
-	}
-
-	if err := req.ParseForm(); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-	if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_QueryService_GetTrace_0); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-
-	stream, err := client.GetTrace(ctx, &protoReq)
-	if err != nil {
-		return nil, metadata, err
-	}
-	header, err := stream.Header()
-	if err != nil {
-		return nil, metadata, err
-	}
-	metadata.HeaderMD = header
-	return stream, metadata, nil
-
-}
-
-var (
-	filter_QueryService_FindTraces_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)}
-)
-
-func request_QueryService_FindTraces_0(ctx context.Context, marshaler runtime.Marshaler, client QueryServiceClient, req *http.Request, pathParams map[string]string) (QueryService_FindTracesClient, runtime.ServerMetadata, error) {
-	var protoReq FindTracesRequest
-	var metadata runtime.ServerMetadata
-
-	if err := req.ParseForm(); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-	if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_QueryService_FindTraces_0); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-
-	stream, err := client.FindTraces(ctx, &protoReq)
-	if err != nil {
-		return nil, metadata, err
-	}
-	header, err := stream.Header()
-	if err != nil {
-		return nil, metadata, err
-	}
-	metadata.HeaderMD = header
-	return stream, metadata, nil
-
-}
-
-func request_QueryService_GetServices_0(ctx context.Context, marshaler runtime.Marshaler, client QueryServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
-	var protoReq GetServicesRequest
-	var metadata runtime.ServerMetadata
-
-	msg, err := client.GetServices(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
-	return msg, metadata, err
-
-}
-
-func local_request_QueryService_GetServices_0(ctx context.Context, marshaler runtime.Marshaler, server QueryServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
-	var protoReq GetServicesRequest
-	var metadata runtime.ServerMetadata
-
-	msg, err := server.GetServices(ctx, &protoReq)
-	return msg, metadata, err
-
-}
-
-var (
-	filter_QueryService_GetOperations_0 = &utilities.DoubleArray{Encoding: map[string]int{}, Base: []int(nil), Check: []int(nil)}
-)
-
-func request_QueryService_GetOperations_0(ctx context.Context, marshaler runtime.Marshaler, client QueryServiceClient, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
-	var protoReq GetOperationsRequest
-	var metadata runtime.ServerMetadata
-
-	if err := req.ParseForm(); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-	if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_QueryService_GetOperations_0); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-
-	msg, err := client.GetOperations(ctx, &protoReq, grpc.Header(&metadata.HeaderMD), grpc.Trailer(&metadata.TrailerMD))
-	return msg, metadata, err
-
-}
-
-func local_request_QueryService_GetOperations_0(ctx context.Context, marshaler runtime.Marshaler, server QueryServiceServer, req *http.Request, pathParams map[string]string) (proto.Message, runtime.ServerMetadata, error) {
-	var protoReq GetOperationsRequest
-	var metadata runtime.ServerMetadata
-
-	if err := req.ParseForm(); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-	if err := runtime.PopulateQueryParameters(&protoReq, req.Form, filter_QueryService_GetOperations_0); err != nil {
-		return nil, metadata, status.Errorf(codes.InvalidArgument, "%v", err)
-	}
-
-	msg, err := server.GetOperations(ctx, &protoReq)
-	return msg, metadata, err
-
-}
-
-// RegisterQueryServiceHandlerServer registers the http handlers for service QueryService to "mux".
-// UnaryRPC     :call QueryServiceServer directly.
-// StreamingRPC :currently unsupported pending https://github.com/grpc/grpc-go/issues/906.
-// Note that using this registration option will cause many gRPC library features to stop working. Consider using RegisterQueryServiceHandlerFromEndpoint instead.
-func RegisterQueryServiceHandlerServer(ctx context.Context, mux *runtime.ServeMux, server QueryServiceServer) error {
-
-	mux.Handle("GET", pattern_QueryService_GetTrace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		err := status.Error(codes.Unimplemented, "streaming calls are not yet supported in the in-process transport")
-		_, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-		return
-	})
-
-	mux.Handle("GET", pattern_QueryService_FindTraces_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		err := status.Error(codes.Unimplemented, "streaming calls are not yet supported in the in-process transport")
-		_, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-		return
-	})
-
-	mux.Handle("GET", pattern_QueryService_GetServices_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		ctx, cancel := context.WithCancel(req.Context())
-		defer cancel()
-		var stream runtime.ServerTransportStream
-		ctx = grpc.NewContextWithServerTransportStream(ctx, &stream)
-		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-		resp, md, err := local_request_QueryService_GetServices_0(rctx, inboundMarshaler, server, req, pathParams)
-		md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer())
-		ctx = runtime.NewServerMetadataContext(ctx, md)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-
-		forward_QueryService_GetServices_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
-
-	})
-
-	mux.Handle("GET", pattern_QueryService_GetOperations_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		ctx, cancel := context.WithCancel(req.Context())
-		defer cancel()
-		var stream runtime.ServerTransportStream
-		ctx = grpc.NewContextWithServerTransportStream(ctx, &stream)
-		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		rctx, err := runtime.AnnotateIncomingContext(ctx, mux, req)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-		resp, md, err := local_request_QueryService_GetOperations_0(rctx, inboundMarshaler, server, req, pathParams)
-		md.HeaderMD, md.TrailerMD = metadata.Join(md.HeaderMD, stream.Header()), metadata.Join(md.TrailerMD, stream.Trailer())
-		ctx = runtime.NewServerMetadataContext(ctx, md)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-
-		forward_QueryService_GetOperations_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
-
-	})
-
-	return nil
-}
-
-// RegisterQueryServiceHandlerFromEndpoint is same as RegisterQueryServiceHandler but
-// automatically dials to "endpoint" and closes the connection when "ctx" gets done.
-func RegisterQueryServiceHandlerFromEndpoint(ctx context.Context, mux *runtime.ServeMux, endpoint string, opts []grpc.DialOption) (err error) {
-	conn, err := grpc.Dial(endpoint, opts...)
-	if err != nil {
-		return err
-	}
-	defer func() {
-		if err != nil {
-			if cerr := conn.Close(); cerr != nil {
-				grpclog.Infof("Failed to close conn to %s: %v", endpoint, cerr)
-			}
-			return
-		}
-		go func() {
-			<-ctx.Done()
-			if cerr := conn.Close(); cerr != nil {
-				grpclog.Infof("Failed to close conn to %s: %v", endpoint, cerr)
-			}
-		}()
-	}()
-
-	return RegisterQueryServiceHandler(ctx, mux, conn)
-}
-
-// RegisterQueryServiceHandler registers the http handlers for service QueryService to "mux".
-// The handlers forward requests to the grpc endpoint over "conn".
-func RegisterQueryServiceHandler(ctx context.Context, mux *runtime.ServeMux, conn *grpc.ClientConn) error {
-	return RegisterQueryServiceHandlerClient(ctx, mux, NewQueryServiceClient(conn))
-}
-
-// RegisterQueryServiceHandlerClient registers the http handlers for service QueryService
-// to "mux". The handlers forward requests to the grpc endpoint over the given implementation of "QueryServiceClient".
-// Note: the gRPC framework executes interceptors within the gRPC handler. If the passed in "QueryServiceClient"
-// doesn't go through the normal gRPC flow (creating a gRPC client etc.) then it will be up to the passed in
-// "QueryServiceClient" to call the correct interceptors.
-func RegisterQueryServiceHandlerClient(ctx context.Context, mux *runtime.ServeMux, client QueryServiceClient) error {
-
-	mux.Handle("GET", pattern_QueryService_GetTrace_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		ctx, cancel := context.WithCancel(req.Context())
-		defer cancel()
-		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		rctx, err := runtime.AnnotateContext(ctx, mux, req)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-		resp, md, err := request_QueryService_GetTrace_0(rctx, inboundMarshaler, client, req, pathParams)
-		ctx = runtime.NewServerMetadataContext(ctx, md)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-
-		forward_QueryService_GetTrace_0(ctx, mux, outboundMarshaler, w, req, func() (proto.Message, error) { return resp.Recv() }, mux.GetForwardResponseOptions()...)
-
-	})
-
-	mux.Handle("GET", pattern_QueryService_FindTraces_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		ctx, cancel := context.WithCancel(req.Context())
-		defer cancel()
-		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		rctx, err := runtime.AnnotateContext(ctx, mux, req)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-		resp, md, err := request_QueryService_FindTraces_0(rctx, inboundMarshaler, client, req, pathParams)
-		ctx = runtime.NewServerMetadataContext(ctx, md)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-
-		forward_QueryService_FindTraces_0(ctx, mux, outboundMarshaler, w, req, func() (proto.Message, error) { return resp.Recv() }, mux.GetForwardResponseOptions()...)
-
-	})
-
-	mux.Handle("GET", pattern_QueryService_GetServices_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		ctx, cancel := context.WithCancel(req.Context())
-		defer cancel()
-		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		rctx, err := runtime.AnnotateContext(ctx, mux, req)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-		resp, md, err := request_QueryService_GetServices_0(rctx, inboundMarshaler, client, req, pathParams)
-		ctx = runtime.NewServerMetadataContext(ctx, md)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-
-		forward_QueryService_GetServices_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
-
-	})
-
-	mux.Handle("GET", pattern_QueryService_GetOperations_0, func(w http.ResponseWriter, req *http.Request, pathParams map[string]string) {
-		ctx, cancel := context.WithCancel(req.Context())
-		defer cancel()
-		inboundMarshaler, outboundMarshaler := runtime.MarshalerForRequest(mux, req)
-		rctx, err := runtime.AnnotateContext(ctx, mux, req)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-		resp, md, err := request_QueryService_GetOperations_0(rctx, inboundMarshaler, client, req, pathParams)
-		ctx = runtime.NewServerMetadataContext(ctx, md)
-		if err != nil {
-			runtime.HTTPError(ctx, mux, outboundMarshaler, w, req, err)
-			return
-		}
-
-		forward_QueryService_GetOperations_0(ctx, mux, outboundMarshaler, w, req, resp, mux.GetForwardResponseOptions()...)
-
-	})
-
-	return nil
-}
-
-var (
-	pattern_QueryService_GetTrace_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2, 1, 0, 4, 1, 5, 3}, []string{"api", "v3", "traces", "trace_id"}, "", runtime.AssumeColonVerbOpt(true)))
-
-	pattern_QueryService_FindTraces_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v3", "traces"}, "", runtime.AssumeColonVerbOpt(true)))
-
-	pattern_QueryService_GetServices_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v3", "services"}, "", runtime.AssumeColonVerbOpt(true)))
-
-	pattern_QueryService_GetOperations_0 = runtime.MustPattern(runtime.NewPattern(1, []int{2, 0, 2, 1, 2, 2}, []string{"api", "v3", "operations"}, "", runtime.AssumeColonVerbOpt(true)))
-)
-
-var (
-	forward_QueryService_GetTrace_0 = runtime.ForwardResponseStream
-
-	forward_QueryService_FindTraces_0 = runtime.ForwardResponseStream
-
-	forward_QueryService_GetServices_0 = runtime.ForwardResponseMessage
-
-	forward_QueryService_GetOperations_0 = runtime.ForwardResponseMessage
-)

From 5c6348e171a2c7a42949e0495675b2bbb02bc6f5 Mon Sep 17 00:00:00 2001
From: Harshvir Potpose <122517264+akagami-harsh@users.noreply.github.com>
Date: Sun, 31 Dec 2023 02:27:23 +0530
Subject: [PATCH 2/2] Fix Goroutine leaks in several packages (#5066)

## Which problem is this PR solving?
- fix gorourine leak in some packages

## Description of the changes
- part of #5006

## How was this change tested?
- go test

## Checklist
- [x] I have read
https://github.com/jaegertracing/jaeger/blob/master/CONTRIBUTING_GUIDELINES.md
- [x] I have signed all commits
- [x] I have run lint and test steps successfully
  - for `jaeger`: `make lint test`

---------

Signed-off-by: Harshvir Potpose <hpotpose62@gmail.com>
---
 cmd/ingester/app/consumer/package_test.go     | 25 +++++++++++++++++
 .../app/consumer/processor_factory_test.go    |  1 +
 cmd/internal/env/command_test.go              |  7 +++++
 pkg/testutils/leakcheck.go                    | 27 +++++++++++++++++++
 pkg/testutils/leakcheck_test.go               | 26 ++++++++++++++++++
 .../badger/dependencystore/package_test.go    | 27 +++++++++++++++++++
 plugin/storage/badger/package_test.go         | 27 +++++++++++++++++++
 .../badger/samplingstore/storage_test.go      |  6 +++++
 .../storage/badger/spanstore/package_test.go  | 27 +++++++++++++++++++
 plugin/storage/factory_test.go                |  2 ++
 plugin/storage/package_test.go                | 27 +++++++++++++++++++
 11 files changed, 202 insertions(+)
 create mode 100644 cmd/ingester/app/consumer/package_test.go
 create mode 100644 pkg/testutils/leakcheck.go
 create mode 100644 pkg/testutils/leakcheck_test.go
 create mode 100644 plugin/storage/badger/dependencystore/package_test.go
 create mode 100644 plugin/storage/badger/package_test.go
 create mode 100644 plugin/storage/badger/spanstore/package_test.go
 create mode 100644 plugin/storage/package_test.go

diff --git a/cmd/ingester/app/consumer/package_test.go b/cmd/ingester/app/consumer/package_test.go
new file mode 100644
index 00000000000..287cb02e5b4
--- /dev/null
+++ b/cmd/ingester/app/consumer/package_test.go
@@ -0,0 +1,25 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 consumer
+
+import (
+	"testing"
+
+	"go.uber.org/goleak"
+)
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m)
+}
diff --git a/cmd/ingester/app/consumer/processor_factory_test.go b/cmd/ingester/app/consumer/processor_factory_test.go
index 5ea0fe8e7b8..f64721e8e76 100644
--- a/cmd/ingester/app/consumer/processor_factory_test.go
+++ b/cmd/ingester/app/consumer/processor_factory_test.go
@@ -55,6 +55,7 @@ func Test_new(t *testing.T) {
 	}
 
 	processor := pf.new(topic, partition, offset)
+	defer processor.Close()
 	msg := &kmocks.Message{}
 	msg.On("Offset").Return(offset + 1)
 	processor.Process(msg)
diff --git a/cmd/internal/env/command_test.go b/cmd/internal/env/command_test.go
index 20baed3f423..4169b70eeae 100644
--- a/cmd/internal/env/command_test.go
+++ b/cmd/internal/env/command_test.go
@@ -20,6 +20,9 @@ import (
 	"testing"
 
 	"github.com/stretchr/testify/assert"
+	"go.uber.org/goleak"
+
+	"github.com/jaegertracing/jaeger/pkg/testutils"
 )
 
 func TestCommand(t *testing.T) {
@@ -30,3 +33,7 @@ func TestCommand(t *testing.T) {
 	assert.True(t, strings.Contains(buf.String(), "METRICS_BACKEND"))
 	assert.True(t, strings.Contains(buf.String(), "SPAN_STORAGE"))
 }
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m, testutils.IgnoreGlogFlushDaemonLeak())
+}
diff --git a/pkg/testutils/leakcheck.go b/pkg/testutils/leakcheck.go
new file mode 100644
index 00000000000..b5e25503382
--- /dev/null
+++ b/pkg/testutils/leakcheck.go
@@ -0,0 +1,27 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 testutils
+
+import (
+	"go.uber.org/goleak"
+)
+
+// IgnoreGlogFlushDaemonLeak returns a goleak.Option that ignores the flushDaemon function
+// from the glog package that can cause false positives in leak detection.
+// This is necessary because glog starts a goroutine in the background that may not
+// be stopped when the test finishes, leading to a detected but expected leak.
+func IgnoreGlogFlushDaemonLeak() goleak.Option {
+	return goleak.IgnoreTopFunction("github.com/golang/glog.(*fileSink).flushDaemon")
+}
diff --git a/pkg/testutils/leakcheck_test.go b/pkg/testutils/leakcheck_test.go
new file mode 100644
index 00000000000..4fa06e9f8cd
--- /dev/null
+++ b/pkg/testutils/leakcheck_test.go
@@ -0,0 +1,26 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 testutils
+
+import (
+	"testing"
+)
+
+func TestIgnoreGlogFlushDaemonLeak(t *testing.T) {
+	opt := IgnoreGlogFlushDaemonLeak()
+	if opt == nil {
+		t.Errorf("IgnoreGlogFlushDaemonLeak() returned nil, want non-nil goleak.Option")
+	}
+}
diff --git a/plugin/storage/badger/dependencystore/package_test.go b/plugin/storage/badger/dependencystore/package_test.go
new file mode 100644
index 00000000000..a2d168f1f48
--- /dev/null
+++ b/plugin/storage/badger/dependencystore/package_test.go
@@ -0,0 +1,27 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 dependencystore
+
+import (
+	"testing"
+
+	"go.uber.org/goleak"
+
+	"github.com/jaegertracing/jaeger/pkg/testutils"
+)
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m, testutils.IgnoreGlogFlushDaemonLeak())
+}
diff --git a/plugin/storage/badger/package_test.go b/plugin/storage/badger/package_test.go
new file mode 100644
index 00000000000..6fc584128b3
--- /dev/null
+++ b/plugin/storage/badger/package_test.go
@@ -0,0 +1,27 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 badger
+
+import (
+	"testing"
+
+	"go.uber.org/goleak"
+
+	"github.com/jaegertracing/jaeger/pkg/testutils"
+)
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m, testutils.IgnoreGlogFlushDaemonLeak())
+}
diff --git a/plugin/storage/badger/samplingstore/storage_test.go b/plugin/storage/badger/samplingstore/storage_test.go
index 1e0bedc1ca0..8296f966c92 100644
--- a/plugin/storage/badger/samplingstore/storage_test.go
+++ b/plugin/storage/badger/samplingstore/storage_test.go
@@ -22,8 +22,10 @@ import (
 	"github.com/dgraph-io/badger/v3"
 	"github.com/stretchr/testify/assert"
 	"github.com/stretchr/testify/require"
+	"go.uber.org/goleak"
 
 	samplemodel "github.com/jaegertracing/jaeger/cmd/collector/app/sampling/model"
+	"github.com/jaegertracing/jaeger/pkg/testutils"
 )
 
 func newTestSamplingStore(db *badger.DB) *SamplingStore {
@@ -139,3 +141,7 @@ func runWithBadger(t *testing.T, test func(t *testing.T, store *SamplingStore))
 	ss := newTestSamplingStore(store)
 	test(t, ss)
 }
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m, testutils.IgnoreGlogFlushDaemonLeak())
+}
diff --git a/plugin/storage/badger/spanstore/package_test.go b/plugin/storage/badger/spanstore/package_test.go
new file mode 100644
index 00000000000..a601b5f157b
--- /dev/null
+++ b/plugin/storage/badger/spanstore/package_test.go
@@ -0,0 +1,27 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 spanstore
+
+import (
+	"testing"
+
+	"go.uber.org/goleak"
+
+	"github.com/jaegertracing/jaeger/pkg/testutils"
+)
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m, testutils.IgnoreGlogFlushDaemonLeak())
+}
diff --git a/plugin/storage/factory_test.go b/plugin/storage/factory_test.go
index c82b9f03dca..c5220e0c3e1 100644
--- a/plugin/storage/factory_test.go
+++ b/plugin/storage/factory_test.go
@@ -411,7 +411,9 @@ func TestPublishOpts(t *testing.T) {
 	require.NoError(t, err)
 
 	baseMetrics := metricstest.NewFactory(time.Second)
+	baseMetrics.Stop()
 	forkFactory := metricstest.NewFactory(time.Second)
+	forkFactory.Stop()
 	metricsFactory := fork.New("internal", forkFactory, baseMetrics)
 	f.metricsFactory = metricsFactory
 
diff --git a/plugin/storage/package_test.go b/plugin/storage/package_test.go
new file mode 100644
index 00000000000..7627d753b67
--- /dev/null
+++ b/plugin/storage/package_test.go
@@ -0,0 +1,27 @@
+// Copyright (c) 2023 The Jaeger Authors.
+//
+// Licensed 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 storage
+
+import (
+	"testing"
+
+	"go.uber.org/goleak"
+
+	"github.com/jaegertracing/jaeger/pkg/testutils"
+)
+
+func TestMain(m *testing.M) {
+	goleak.VerifyTestMain(m, testutils.IgnoreGlogFlushDaemonLeak())
+}