Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
59146: bazel: generate `//go:generate stringer` files sandbox r=rickystewart a=alan-mas

First part of #57787 work.

As `gazelle` is not taking care by itself in any changes related to `//go:generate stringer` we are creating a
`genrule` to handle it.

From all the `//go:generate stringer` files, there are some that are having troubles during bazel build:
```
-pkg/util/encoding/encoding.go
-pkg/util/encoding/BUILD.bazel
"stringer: can't handle non-integer constant type Type"

-pkg/workload/schemachange/schemachange.go
-pkg/workload/schemachange/BUILD.bazel
"stringer: can't happen: constant is not an integer TxStatusInFailure"
```

Release note: None

59327: build,bazel: inject nodejs support into our bazel setup r=rickystewart a=rickystewart

Use the standard Google-supported Bazel tools to do so. This isn't
plugged into anything yet, but you can verify that it works with `bazel
query @npm//...`.

Release note: None

59350: server: wait for SQL readiness in the `/health?ready=1` probe r=joshimhoff a=knz

(smaller version than #59191) 
Fixes #58864.
Planning to backport to v20.2.

Release note (api change): the health API now checks that the SQL
server is ready to accept clients when a readiness check is requested.

Co-authored-by: Alanmas <[email protected]>
Co-authored-by: Ricky Stewart <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
  • Loading branch information
4 people committed Jan 25, 2021
4 parents fad50c1 + ef93819 + 6882bb5 + 3f0c542 commit bf439cd
Show file tree
Hide file tree
Showing 29 changed files with 390 additions and 5 deletions.
21 changes: 20 additions & 1 deletion WORKSPACE
Original file line number Diff line number Diff line change
@@ -1,6 +1,9 @@
# Define the top level namespace. This lets everything be addressable using
# `@cockroach//...`.
workspace(name = "cockroach")
workspace(
name = "cockroach",
managed_directories = {"@npm": ["node_modules"]},
)

# Load the things that let us load other things.
load("@bazel_tools//tools/build_defs/repo:http.bzl", "http_archive")
Expand All @@ -16,6 +19,13 @@ http_archive(
],
)

# Like the above, but for nodeJS.
http_archive(
name = "build_bazel_rules_nodejs",
sha256 = "6142e9586162b179fdd570a55e50d1332e7d9c030efd853453438d607569721d",
urls = ["https://github.com/bazelbuild/rules_nodejs/releases/download/3.0.0/rules_nodejs-3.0.0.tar.gz"],
)

# Load gazelle. This lets us auto-generate BUILD.bazel files throughout the
# repo.
#
Expand Down Expand Up @@ -82,6 +92,15 @@ go_rules_dependencies()

go_register_toolchains(go_version = "1.15.6")

# Configure nodeJS.
load("@build_bazel_rules_nodejs//:index.bzl", "yarn_install")

yarn_install(
name = "npm",
package_json = "//pkg/ui:package.json",
yarn_lock = "//pkg/ui:yarn.lock",
)

# NB: @bazel_skylib comes from go_rules_dependencies().
load("@bazel_skylib//lib:versions.bzl", "versions")

Expand Down
18 changes: 18 additions & 0 deletions pkg/STRINGER.bzl
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
# Common function to create //go:generate stringer files within bazel sandbox

def stringer(file, typ, name):
native.genrule(
name = name,
srcs = [
file,
],
outs = [typ.lower() + "_string.go"],
cmd = """
env PATH=`dirname $(location @go_sdk//:bin/go)` HOME=$(GENDIR) \
$(location @org_golang_x_tools//cmd/stringer:stringer) -output=$@ -type={} $<
""".format(typ),
tools = [
"@go_sdk//:bin/go",
"@org_golang_x_tools//cmd/stringer",
],
)
8 changes: 8 additions & 0 deletions pkg/base/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "base",
Expand All @@ -14,6 +15,7 @@ go_library(
"testclusterreplicationmode_string.go",
"testing_knobs.go",
"zone.go",
":gen-clusterreplication-stringer", # keep
],
importpath = "github.com/cockroachdb/cockroach/pkg/base",
visibility = ["//visibility:public"],
Expand Down Expand Up @@ -64,3 +66,9 @@ go_test(
"@com_github_stretchr_testify//require",
],
)

stringer(
name = "gen-clusterreplication-stringer",
file = "test_server_args.go",
typ = "TestClusterReplicationMode",
)
8 changes: 8 additions & 0 deletions pkg/ccl/sqlproxyccl/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "sqlproxyccl",
Expand All @@ -12,6 +13,7 @@ go_library(
"metrics.go",
"proxy.go",
"server.go",
":gen-errorcode-stringer", # keep
],
importpath = "github.com/cockroachdb/cockroach/pkg/ccl/sqlproxyccl",
visibility = ["//visibility:public"],
Expand Down Expand Up @@ -58,3 +60,9 @@ go_test(
"@com_github_stretchr_testify//require",
],
)

stringer(
name = "gen-errorcode-stringer",
file = "error.go",
typ = "ErrorCode",
)
8 changes: 8 additions & 0 deletions pkg/cli/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "cli",
Expand Down Expand Up @@ -51,6 +52,7 @@ go_library(
"tsdump.go",
"userfile.go",
"zip.go",
":gen-keytype-stringer", # keep
],
# keep
cdeps = [
Expand Down Expand Up @@ -336,3 +338,9 @@ go_test(
"@com_github_stretchr_testify//require",
],
)

stringer(
name = "gen-keytype-stringer",
file = "flags_util.go",
typ = "keyType",
)
8 changes: 8 additions & 0 deletions pkg/clusterversion/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
load("@rules_proto//proto:defs.bzl", "proto_library")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "clusterversion",
Expand All @@ -11,6 +12,7 @@ go_library(
"keyed_versions.go",
"setting.go",
"testutils.go",
":gen-key-stringer", # keep
],
embed = [":clusterversion_go_proto"],
importpath = "github.com/cockroachdb/cockroach/pkg/clusterversion",
Expand Down Expand Up @@ -61,3 +63,9 @@ go_proto_library(
"@com_github_gogo_protobuf//gogoproto",
],
)

stringer(
name = "gen-key-stringer",
file = "cockroach_versions.go",
typ = "Key",
)
8 changes: 8 additions & 0 deletions pkg/kv/kvclient/kvcoord/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "kvcoord",
Expand Down Expand Up @@ -28,6 +29,7 @@ go_library(
"txn_lock_gatekeeper.go",
"txn_metrics.go",
"txnstate_string.go",
":gen-txnstate-stringer", # keep
],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvclient/kvcoord",
visibility = ["//visibility:public"],
Expand Down Expand Up @@ -149,3 +151,9 @@ go_test(
"@org_golang_x_sync//errgroup",
],
)

stringer(
name = "gen-txnstate-stringer",
file = "txn_coord_sender.go",
typ = "txnState",
)
8 changes: 8 additions & 0 deletions pkg/kv/kvserver/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
load("@rules_proto//proto:defs.bzl", "proto_library")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "kvserver",
Expand Down Expand Up @@ -91,6 +92,7 @@ go_library(
"testing_knobs.go",
"track_raft_protos.go",
"ts_maintenance_queue.go",
":gen-refreshraftreason-stringer", # keep
],
embed = [":kvserver_go_proto"],
importpath = "github.com/cockroachdb/cockroach/pkg/kv/kvserver",
Expand Down Expand Up @@ -427,3 +429,9 @@ go_proto_library(
"@io_etcd_go_etcd_raft_v3//raftpb",
],
)

stringer(
name = "gen-refreshraftreason-stringer",
file = "replica_raft.go",
typ = "refreshRaftReason",
)
17 changes: 17 additions & 0 deletions pkg/roachpb/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,6 +1,7 @@
load("@rules_proto//proto:defs.bzl", "proto_library")
load("@io_bazel_rules_go//proto:def.bzl", "go_proto_library")
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "roachpb",
Expand All @@ -21,6 +22,8 @@ go_library(
"span_group.go",
"tenant.go",
"version.go",
":gen-errordetailtype-stringer", # keep
":gen-method-stringer", # keep
],
embed = [":roachpb_go_proto"],
importpath = "github.com/cockroachdb/cockroach/pkg/roachpb",
Expand Down Expand Up @@ -146,3 +149,17 @@ go_proto_library(
"@com_github_gogo_protobuf//gogoproto",
],
)

# Using common function for stringer to create method_string.go
stringer(
name = "gen-method-stringer",
file = "method.go",
typ = "Method",
)

# Using common function for stringer to create errordetailtype_string.go
stringer(
name = "gen-errordetailtype-stringer",
file = "errors.go",
typ = "ErrorDetailType",
)
8 changes: 6 additions & 2 deletions pkg/server/admin.go
Original file line number Diff line number Diff line change
Expand Up @@ -1358,13 +1358,13 @@ func (s *adminServer) Health(
return resp, nil
}

if err := s.checkReadinessForHealthCheck(); err != nil {
if err := s.checkReadinessForHealthCheck(ctx); err != nil {
return nil, err
}
return resp, nil
}

func (s *adminServer) checkReadinessForHealthCheck() error {
func (s *adminServer) checkReadinessForHealthCheck(ctx context.Context) error {
serveMode := s.server.grpc.mode.get()
switch serveMode {
case modeInitializing:
Expand Down Expand Up @@ -1397,6 +1397,10 @@ func (s *adminServer) checkReadinessForHealthCheck() error {
return status.Errorf(codes.Unavailable, "node is shutting down")
}

if !s.server.sqlServer.acceptingClients.Get() {
return status.Errorf(codes.Unavailable, "node is not accepting SQL clients")
}

return nil
}

Expand Down
16 changes: 14 additions & 2 deletions pkg/server/admin_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1355,6 +1355,7 @@ func TestHealthAPI(t *testing.T) {

s, _, _ := serverutils.StartServer(t, base.TestServerArgs{})
defer s.Stopper().Stop(ctx)
ts := s.(*TestServer)

// We need to retry because the node ID isn't set until after
// bootstrapping.
Expand All @@ -1363,15 +1364,26 @@ func TestHealthAPI(t *testing.T) {
return getAdminJSONProto(s, "health", &resp)
})

// Make the SQL listener appear unavailable. Verify that health fails after that.
ts.sqlServer.acceptingClients.Set(false)
var resp serverpb.HealthResponse
err := getAdminJSONProto(s, "health?ready=1", &resp)
if err == nil {
t.Error("server appears ready even though SQL listener is not")
}
ts.sqlServer.acceptingClients.Set(true)
err = getAdminJSONProto(s, "health?ready=1", &resp)
if err != nil {
t.Errorf("server not ready after SQL listener is ready again: %v", err)
}

// Expire this node's liveness record by pausing heartbeats and advancing the
// server's clock.
ts := s.(*TestServer)
defer ts.nodeLiveness.PauseAllHeartbeatsForTest()()
self, ok := ts.nodeLiveness.Self()
assert.True(t, ok)
s.Clock().Update(self.Expiration.ToTimestamp().Add(1, 0).UnsafeToClockTimestamp())

var resp serverpb.HealthResponse
testutils.SucceedsSoon(t, func() error {
err := getAdminJSONProto(s, "health?ready=1", &resp)
if err == nil {
Expand Down
1 change: 1 addition & 0 deletions pkg/server/drain.go
Original file line number Diff line number Diff line change
Expand Up @@ -188,6 +188,7 @@ func (s *Server) drainClients(ctx context.Context, reporter func(int, redact.Saf
// Mark the server as draining in a way that probes to
// /health?ready=1 will notice.
s.grpc.setMode(modeDraining)
s.sqlServer.acceptingClients.Set(false)
// Wait for drainUnreadyWait. This will fail load balancer checks and
// delay draining so that client traffic can move off this node.
time.Sleep(drainWait.Get(&s.st.SV))
Expand Down
3 changes: 3 additions & 0 deletions pkg/server/server.go
Original file line number Diff line number Diff line change
Expand Up @@ -2073,6 +2073,9 @@ func (s *SQLServer) startServeSQL(
return err
}
}

s.acceptingClients.Set(true)

return nil
}

Expand Down
5 changes: 5 additions & 0 deletions pkg/server/server_sql.go
Original file line number Diff line number Diff line change
Expand Up @@ -70,6 +70,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/mon"
"github.com/cockroachdb/cockroach/pkg/util/netutil"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/marusama/semaphore"
Expand Down Expand Up @@ -111,6 +112,10 @@ type SQLServer struct {
// connManager is the connection manager to use to set up additional
// SQL listeners in AcceptClients().
connManager netutil.Server

// set to true when the server has started accepting client conns.
// Used by health checks.
acceptingClients syncutil.AtomicBool
}

// sqlServerOptionalKVArgs are the arguments supplied to newSQLServer which are
Expand Down
29 changes: 29 additions & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
@@ -1,4 +1,5 @@
load("@io_bazel_rules_go//go:def.bzl", "go_library", "go_test")
load("//pkg:STRINGER.bzl", "stringer")

go_library(
name = "sql",
Expand Down Expand Up @@ -218,6 +219,10 @@ go_library(
"zero.go",
"zigzag_join.go",
"zone_config.go",
":gen-advancecode-stringer", # keep
":gen-nodestatus-stringer", # keep
":gen-txnevent-stringer", # keep
":gen-txntype-stringer", # keep
],
importpath = "github.com/cockroachdb/cockroach/pkg/sql",
visibility = ["//visibility:public"],
Expand Down Expand Up @@ -588,3 +593,27 @@ go_test(
"@org_golang_x_sync//errgroup",
],
)

stringer(
name = "gen-txnevent-stringer",
file = "txn_state.go",
typ = "txnEvent",
)

stringer(
name = "gen-txntype-stringer",
file = "txn_state.go",
typ = "txnType",
)

stringer(
name = "gen-advancecode-stringer",
file = "txn_state.go",
typ = "advanceCode",
)

stringer(
name = "gen-nodestatus-stringer",
file = "distsql_physical_planner.go",
typ = "NodeStatus",
)
Loading

0 comments on commit bf439cd

Please sign in to comment.