Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
53842: server: always create a liveness record before starting up r=irfansharif a=irfansharif

Previously it used to be the case that it was possible for a node to be
up and running, and for there to be no corresponding liveness record for
it. This was a very transient situation as liveness records are created
for a given node as soon as it out its first heartbeat. Still, given
that this could take a few seconds, it lent to a lot of complexity in
our handling of node liveness where we had to always anticipate the
possibility of there being no corresponding liveness record for a given
node (and thus creating it if necessary).

Having a liveness record for each node always present is a crucial
building block for long running migrations (#48843). There the intention
is to have the orchestrator process look towards the list of liveness
records for an authoritative view of cluster membership. Previously when
it was possible for an active member of the cluster to not have a
corresponding liveness record (no matter how unlikely or short-lived in
practice), we could not generate such a view.

---

This is an alternative implementation for #53805. Here we choose to
manually write the liveness record for the bootstrapping node when
writing initial cluster data. For all other nodes, we do it on the
server-side of the join RPC. We're also careful to do it in the legacy
codepath when joining a cluster through gossip.

Release note: None


53994: cli: allow SQL commands to use password authn in more cases r=bdarnell,irfansharif,aaron-crl a=knz

First two commits from #53991.

Previously, SQL password authn was only allowed over TLS connections.

With this change, password authn is allowed regardless of whether the
connection uses TLS.

This is implemented by also only asking for a password interactively
the first time that the server complains that pw auth has failed. This
way, no password is ever requested interactively if the server
"trusts" the connection (via HBA rules or `--insecure`).

Release justification: low risk, high benefit changes to existing functionality


54035: sql: emit more tracing events from the stats cache r=RaduBerinde a=RaduBerinde

The stats cache has various "slow" paths (where we need to query the
system table). These are currently only logged if verbosity is high.

This change switches to `VEvent` in most cases, so that these are
visible during tracing (including in statement diagnostics bundles).
This will allow us to diagnose slow planning times, e.g. due to the
stats cache getting full.

Release justification: low-risk change to existing functionality, high
potential benefit for debugging issues.

Release note: None

Co-authored-by: irfan sharif <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
4 people committed Sep 10, 2020
4 parents e8fe416 + 81c9c91 + 57d264f + 7ea3a55 commit 6227db0
Show file tree
Hide file tree
Showing 21 changed files with 345 additions and 104 deletions.
9 changes: 7 additions & 2 deletions pkg/cli/client_url.go
Original file line number Diff line number Diff line change
Expand Up @@ -190,8 +190,13 @@ func (u urlParser) setInternal(v string, warn bool) error {

switch sslMode := options.Get("sslmode"); sslMode {
case "", "disable":
if err := fl.Set(cliflags.ClientInsecure.Name, "true"); err != nil {
return errors.Wrapf(err, "setting insecure connection based on --url")
if u.sslStrict {
// For "strict" mode (RPC client commands) we don't support non-TLS
// yet. See https://github.com/cockroachdb/cockroach/issues/54007
// Instead, we see a request for no TLS to imply insecure mode.
if err := fl.Set(cliflags.ClientInsecure.Name, "true"); err != nil {
return errors.Wrapf(err, "setting secure connection based on --url")
}
}
case "require", "verify-ca", "verify-full":
if sslMode != "verify-full" && u.sslStrict {
Expand Down
9 changes: 9 additions & 0 deletions pkg/cli/context.go
Original file line number Diff line number Diff line change
Expand Up @@ -154,6 +154,14 @@ type cliContext struct {
// extraConnURLOptions contains any additional query URL options
// specified in --url that do not have discrete equivalents.
extraConnURLOptions url.Values

// allowUnencryptedClientPassword enables the CLI commands to use
// password authentication over non-TLS TCP connections. This is
// disallowed by default: the user must opt-in and understand that
// CockroachDB does not guarantee confidentiality of a password
// provided this way.
// TODO(knz): Relax this when SCRAM is implemented.
allowUnencryptedClientPassword bool
}

// cliCtx captures the command-line parameters common to most CLI utilities.
Expand Down Expand Up @@ -184,6 +192,7 @@ func setCliContextDefaults() {
cliCtx.sqlConnPasswd = ""
cliCtx.sqlConnDBName = ""
cliCtx.extraConnURLOptions = nil
cliCtx.allowUnencryptedClientPassword = false
}

// sqlCtx captures the command-line parameters of the `sql` command.
Expand Down
1 change: 0 additions & 1 deletion pkg/cli/flags.go
Original file line number Diff line number Diff line change
Expand Up @@ -692,7 +692,6 @@ func init() {
_ = f.MarkHidden(cliflags.ClientHost.Name)
stringFlag(f, &cliCtx.clientConnPort, cliflags.ClientPort)
_ = f.MarkHidden(cliflags.ClientPort.Name)

}

if cmd == sqlShellCmd {
Expand Down
35 changes: 27 additions & 8 deletions pkg/cli/flags_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -274,8 +274,12 @@ func TestClientURLFlagEquivalence(t *testing.T) {
{anyNonSQL, []string{"--url=postgresql://b:12345"}, []string{"--host=b", "--port=12345"}, "", ""},
{anyNonSQL, []string{"--url=postgresql://b:c"}, nil, `invalid port ":c" after host`, ""},

{anyCmd, []string{"--url=postgresql://foo?application_name=abc"}, []string{"--host=foo", "--insecure"}, "", ""},
{anyCmd, []string{"--url=postgresql://foo?sslmode=disable"}, []string{"--host=foo", "--insecure"}, "", ""},
{anyNonSQL, []string{"--url=postgresql://foo?application_name=abc"}, []string{"--host=foo", "--insecure"}, "", ""},
{anySQL, []string{"--url=postgresql://foo?application_name=abc"}, []string{"--host=foo"}, "", ""},

{anyNonSQL, []string{"--url=postgresql://foo?sslmode=disable"}, []string{"--host=foo", "--insecure"}, "", ""},
{anySQL, []string{"--url=postgresql://foo?sslmode=disable"}, []string{"--host=foo"}, "", ""},

{anySQL, []string{"--url=postgresql://foo?sslmode=require"}, []string{"--host=foo", "--insecure=false"}, "", ""},
{anyNonSQL, []string{"--url=postgresql://foo?sslmode=require"}, nil, "command .* only supports sslmode=disable or sslmode=verify-full", ""},
{anyCmd, []string{"--url=postgresql://foo?sslmode=verify-full"}, []string{"--host=foo", "--insecure=false"}, "", ""},
Expand All @@ -287,17 +291,22 @@ func TestClientURLFlagEquivalence(t *testing.T) {
{anyCmd, []string{"--port=baz", "--url=postgresql://foo"}, []string{"--host=foo", "--port=baz"}, "", `invalid port ":baz" after host`},
{sqlShell, []string{"--database=baz", "--url=postgresql://foo"}, []string{"--host=foo", "--database=baz"}, "", ""},
{anySQL, []string{"--user=baz", "--url=postgresql://foo"}, []string{"--host=foo", "--user=baz"}, "", ""},

{anyCmd, []string{"--insecure=false", "--url=postgresql://foo"}, []string{"--host=foo", "--insecure=false"}, "", ""},
{anyCmd, []string{"--insecure", "--url=postgresql://foo"}, []string{"--host=foo", "--insecure"}, "", ""},
// Only non-SQL lets --insecure bleed into a URL that does not specify sslmode.
{anyNonSQL, []string{"--insecure", "--url=postgresql://foo"}, []string{"--host=foo", "--insecure"}, "", ""},

// URL overrides previous flags if component specified.
{anyCmd, []string{"--host=baz", "--url=postgresql://bar"}, []string{"--host=bar"}, "", ""},
{anyCmd, []string{"--port=baz", "--url=postgresql://foo:12345"}, []string{"--host=foo", "--port=12345"}, "", ""},
{anyCmd, []string{"--port=baz", "--url=postgresql://foo:bar"}, nil, `invalid port ":bar" after host`, ""},
{sqlShell, []string{"--database=baz", "--url=postgresql://foo/bar"}, []string{"--host=foo", "--database=bar"}, "", ""},
{anySQL, []string{"--user=baz", "--url=postgresql://bar@foo"}, []string{"--host=foo", "--user=bar"}, "", ""},
{anyCmd, []string{"--insecure=false", "--url=postgresql://foo?sslmode=disable"}, []string{"--host=foo", "--insecure"}, "", ""},

{anyNonSQL, []string{"--insecure=false", "--url=postgresql://foo?sslmode=disable"}, []string{"--host=foo", "--insecure"}, "", ""},
{anyCmd, []string{"--insecure", "--url=postgresql://foo?sslmode=verify-full"}, []string{"--host=foo", "--insecure=false"}, "", ""},
// SQL is special case: specifying sslmode= does not imply insecure mode. So the insecure bit does not get reset.
{anySQL, []string{"--insecure=false", "--url=postgresql://foo?sslmode=disable"}, []string{"--host=foo"}, "", ""},

// Discrete flag overrides URL if specified afterwards.
{anyCmd, []string{"--url=postgresql://bar", "--host=baz"}, []string{"--host=baz"}, "", ""},
Expand All @@ -306,8 +315,9 @@ func TestClientURLFlagEquivalence(t *testing.T) {
{anyCmd, []string{"--url=postgresql://foo:bar", "--port=baz"}, nil, `invalid port ":bar" after host`, ""},
{sqlShell, []string{"--url=postgresql://foo/bar", "--database=baz"}, []string{"--host=foo", "--database=baz"}, "", ""},
{anySQL, []string{"--url=postgresql://bar@foo", "--user=baz"}, []string{"--host=foo", "--user=baz"}, "", ""},
{anyCmd, []string{"--url=postgresql://foo?sslmode=disable", "--insecure=false"}, []string{"--host=foo", "--insecure=false"}, "", ""},
{anyCmd, []string{"--url=postgresql://foo?sslmode=verify-full", "--insecure"}, []string{"--host=foo", "--insecure"}, "", ""},

{anyNonSQL, []string{"--url=postgresql://foo?sslmode=disable", "--insecure=false"}, []string{"--host=foo", "--insecure=false"}, "", ""},
{anyNonSQL, []string{"--url=postgresql://foo?sslmode=verify-full", "--insecure"}, []string{"--host=foo", "--insecure"}, "", ""},

// Check that the certs dir is extracted properly.
{anyNonSQL, []string{"--url=postgresql://foo?sslmode=verify-full&sslrootcert=" + testCertsDirPath + "/ca.crt"}, []string{"--host=foo", "--certs-dir=" + testCertsDirPath}, "", ""},
Expand Down Expand Up @@ -346,9 +356,9 @@ func TestClientURLFlagEquivalence(t *testing.T) {
}
}

for _, test := range testData {
for testNum, test := range testData {
for _, cmdName := range test.cmds {
t.Run(fmt.Sprintf("%s/%s", cmdName, strings.Join(test.flags, " ")), func(t *testing.T) {
t.Run(fmt.Sprintf("%d/%s/%s", testNum+1, cmdName, strings.Join(test.flags, " ")), func(t *testing.T) {
cmd, _, _ := cockroachCmd.Find([]string{cmdName})

// Parse using the URL.
Expand Down Expand Up @@ -388,6 +398,15 @@ func TestClientURLFlagEquivalence(t *testing.T) {
t.Fatalf("mismatch: URL %q parses\n%+v,\ndiscrete parses\n%+v", resultURL, urlParams, discreteParams)
}

// For SQL commands only, test that reconstructing the URL
// from discrete flags yield equivalent connection parameters.
// (RPC commands never reconstruct a URL.)
for _, s := range anyNonSQL {
if cmdName == s {
return
}
}

// Re-parse using the derived URL.
// We'll want to ensure below that the derived URL specifies the same parameters
// (i.e. check makeClientConnURL does its work properly).
Expand Down
2 changes: 0 additions & 2 deletions pkg/cli/interactive_tests/test_url_db_override.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -32,8 +32,6 @@ start_test "Check that the insecure flag overrides the sslmode if URL is already
set ::env(COCKROACH_INSECURE) "false"

spawn $argv sql --url "postgresql://test@localhost:26257?sslmode=verify-full" --certs-dir=$certs_dir -e "select 1"
eexpect "password:"
send "\r"
eexpect "SSL is not enabled on the server"
eexpect eof

Expand Down
2 changes: 0 additions & 2 deletions pkg/cli/interactive_tests/test_url_login.tcl
Original file line number Diff line number Diff line change
Expand Up @@ -26,8 +26,6 @@ system "$argv start-single-node --insecure --pid-file=server_pid --socket-dir=.
$argv sql --insecure -e 'select 1'"

spawn $argv sql --url "postgresql://?host=$mywd&port=26257"
eexpect "Enter password"
send "insecure\r"
eexpect root@
send_eof
eexpect eof
Expand Down
93 changes: 76 additions & 17 deletions pkg/cli/sql_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,9 @@ type sqlConn struct {
conn sqlConnI
reconnecting bool

// passwordMissing is true iff the url is missing a password.
passwordMissing bool

pendingNotices []*pq.Error

// delayNotices, if set, makes notices accumulate for printing
Expand Down Expand Up @@ -152,6 +155,26 @@ func (c *sqlConn) ensureConn() error {
// connections only once instead. The context is only used for dialing.
conn, err := connector.Connect(context.TODO())
if err != nil {
// Connection failed: if the failure is due to a mispresented
// password, we're going to fill the password here.
//
// TODO(knz): CockroachDB servers do not properly fill SQLSTATE
// (28P01) for password auth errors, so we have to "make do"
// with a string match. This should be cleaned up by adding
// the missing code server-side.
errStr := strings.TrimPrefix(err.Error(), "pq: ")
if strings.HasPrefix(errStr, "password authentication failed") && c.passwordMissing {
if pErr := c.fillPassword(); pErr != nil {
return errors.CombineErrors(err, pErr)
}
// Recurse, once. We recurse to ensure that pq.NewConnector
// and ConnectorWithNoticeHandler get called with the new URL.
// The recursion only occurs once because fillPassword()
// resets c.passwordMissing, so we cannot get into this
// conditional a second time.
return c.ensureConn()
}
// Not a password auth error, or password already set. Simply fail.
return wrapConnError(err)
}
if c.reconnecting && c.dbName != "" {
Expand Down Expand Up @@ -665,23 +688,33 @@ func makeSQLClient(appName string, defaultMode defaultSQLDb) (*sqlConn, error) {
return nil, err
}

// Insecure connections are insecure and should never see a password. Reject
// one that may be present in the URL already.
if options.Get("sslmode") == "disable" {
if _, pwdSet := baseURL.User.Password(); pwdSet {
return nil, errors.Errorf("cannot specify a password in URL with an insecure connection")
// tcpConn is true iff the connection is going over the network.
tcpConn := baseURL.Host != ""

// If there is no TLS mode yet, conjure one based on defaults.
if options.Get("sslmode") == "" {
if cliCtx.Insecure {
options.Set("sslmode", "disable")
} else if tcpConn {
options.Set("sslmode", "verify-full")
}
} else {
if options.Get("sslcert") == "" || options.Get("sslkey") == "" {
// If there's no password in the URL yet and we don't have a client
// certificate, ask for it and populate it in the URL.
if _, pwdSet := baseURL.User.Password(); !pwdSet {
pwd, err := security.PromptForPassword()
if err != nil {
return nil, err
}
baseURL.User = url.UserPassword(baseURL.User.Username(), pwd)
}
// (We don't use TLS over unix socket conns.)
}

// Prevent explicit TLS request in insecure mode.
if cliCtx.Insecure && options.Get("sslmode") != "disable" {
return nil, errors.Errorf("cannot use TLS connections in insecure mode")
}

// How we're going to authenticate.
_, pwdSet := baseURL.User.Password()
if pwdSet {
// There's a password already configured.

// In insecure mode, we don't want the user to get the mistaken
// idea that a password is worth anything.
if cliCtx.Insecure {
return nil, errors.Errorf("password authentication not enabled in insecure mode")
}
}

Expand All @@ -707,7 +740,33 @@ func makeSQLClient(appName string, defaultMode defaultSQLDb) (*sqlConn, error) {
log.Infof(context.Background(), "connecting with URL: %s", sqlURL)
}

return makeSQLConn(sqlURL), nil
conn := makeSQLConn(sqlURL)

conn.passwordMissing = !pwdSet

return conn, nil
}

// fillPassword is called the first time the server complains that the
// password authentication has failed, if no password was supplied to
// start with. It asks the user for a password interactively.
func (c *sqlConn) fillPassword() error {
connURL, err := url.Parse(c.url)
if err != nil {
return err
}

// Password can be safely encrypted, or the user opted in
// manually to non-encryption. All good.

pwd, err := security.PromptForPassword()
if err != nil {
return err
}
connURL.User = url.UserPassword(connURL.User.Username(), pwd)
c.url = connURL.String()
c.passwordMissing = false
return nil
}

type queryFunc func(conn *sqlConn) (rows *sqlRows, isMultiStatementQuery bool, err error)
Expand Down
15 changes: 0 additions & 15 deletions pkg/cmd/roachtest/decommission.go
Original file line number Diff line number Diff line change
Expand Up @@ -309,21 +309,6 @@ func runDecommissionRandomized(ctx context.Context, t *test, c *cluster) {
Multiplier: 2,
}

// This is a pretty gross hack to let the bootstrap info (cluster ID,
// liveness records) disseminate through the cluster. Since it's no longer
// happening through gossip, it takes a bit longer to happen. We should do
// two things to improve our story here:
//
// - We should opportunistically write to the liveness table when adding a
// node through the Join RPC. This would also simplify the handling of
// empty liveness records (they would no longer exist).
// - We should add roachtest helpers that wait until each node has received
// cluster ID information, and use it in all the tests that need it (which
// may very well be all the tests).
//
// TODO(irfansharif): Do the above.
time.Sleep(30 * time.Second)

// Partially decommission then recommission a random node, from another
// random node. Run a couple of status checks while doing so.
{
Expand Down
9 changes: 9 additions & 0 deletions pkg/kv/kvserver/client_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -1009,6 +1009,15 @@ func (m *multiTestContext) addStore(idx int) {
}{
ch: make(chan struct{}),
}
if idx != 0 {
// Given multiTestContext does not make use of the join RPC, we have to
// manually write out liveness records for each node to maintain the
// invariant that all nodes have liveness records present before they
// start heartbeating.
if err := m.nodeLivenesses[idx].CreateLivenessRecord(ctx, nodeID); err != nil {
m.t.Fatal(err)
}
}
m.nodeLivenesses[idx].StartHeartbeat(ctx, stopper, m.engines[idx:idx+1], func(ctx context.Context) {
now := clock.Now()
if err := store.WriteLastUpTimestamp(ctx, now); err != nil {
Expand Down
Loading

0 comments on commit 6227db0

Please sign in to comment.