Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
48048: util/log,*: enforce literal strings in non-formatting log calls r=irfansharif a=knz

This patch changes the logging interface and the `fmtsafe` linter to enforce that uses of the non-formatting log functions (`log.Info`, `log.Warning` etc) always uses a literal string.

This makes it possible to assume that the argument to those functions
is PII-free and can be collected in PII-free logs and telemetry.



48071: sql: add rule to simplify comparison filters on same vars r=mjibson a=mjibson

Add rules for SELECT WHERE and JOIN ON filters that simplifies comparisons
on same variables to their filter equivalent.

Fixes #47717

Release note: None

48186: opt: handle FKs when dropping tables in testcat r=RaduBerinde a=RaduBerinde

The test catalog was ignoring FKs when dropping tables. We now error out if
there are inbound references, and otherwise clean up the outbound references
from the referenced tables.

Release note: None

Co-authored-by: Raphael 'kena' Poss <knz@thaumogen.net>
Co-authored-by: Matt Jibson <matt.jibson@gmail.com>
Co-authored-by: Radu Berinde <radu@cockroachlabs.com>
  • Loading branch information
4 people committed Apr 30, 2020
4 parents af25bae + c985dfe + 34ebb83 + cdaa010 commit aebcb79
Show file tree
Hide file tree
Showing 130 changed files with 713 additions and 344 deletions.
8 changes: 4 additions & 4 deletions pkg/acceptance/cluster/docker.go
Original file line number Diff line number Diff line change
Expand Up @@ -299,13 +299,13 @@ func (c *Container) Wait(ctx context.Context, condition container.WaitCondition)

out := io.MultiWriter(cmdLog, os.Stderr)
if err := c.Logs(ctx, out); err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
}

if exitCode := waitOKBody.StatusCode; exitCode != 0 {
err = errors.Errorf("non-zero exit code: %d", exitCode)
fmt.Fprintln(out, err.Error())
log.Shout(ctx, log.Severity_INFO, "command left-over files in ", c.cluster.volumesDir)
log.Shoutf(ctx, log.Severity_INFO, "command left-over files in %s", c.cluster.volumesDir)
}

return err
Expand Down Expand Up @@ -351,7 +351,7 @@ func (c *Container) Inspect(ctx context.Context) (types.ContainerJSON, error) {
func (c *Container) Addr(ctx context.Context, port nat.Port) *net.TCPAddr {
containerInfo, err := c.Inspect(ctx)
if err != nil {
log.Error(ctx, err)
log.Errorf(ctx, "%v", err)
return nil
}
bindings, ok := containerInfo.NetworkSettings.Ports[port]
Expand All @@ -360,7 +360,7 @@ func (c *Container) Addr(ctx context.Context, port nat.Port) *net.TCPAddr {
}
portNum, err := strconv.Atoi(bindings[0].HostPort)
if err != nil {
log.Error(ctx, err)
log.Errorf(ctx, "%v", err)
return nil
}
return &net.TCPAddr{
Expand Down
4 changes: 2 additions & 2 deletions pkg/acceptance/cluster/dockercluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -864,15 +864,15 @@ func (l *DockerCluster) ExecCLI(ctx context.Context, i int, cmd []string) (strin
func (l *DockerCluster) Cleanup(ctx context.Context, preserveLogs bool) {
volumes, err := ioutil.ReadDir(l.volumesDir)
if err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
return
}
for _, v := range volumes {
if preserveLogs && v.Name() == "logs" {
continue
}
if err := os.RemoveAll(filepath.Join(l.volumesDir, v.Name())); err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
}
}
}
32 changes: 16 additions & 16 deletions pkg/acceptance/localcluster/cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -302,7 +302,7 @@ func (c *Cluster) makeNode(ctx context.Context, nodeIdx int, cfg NodeConfig) (*N
n.Cfg.ExtraArgs = append(args, cfg.ExtraArgs...)

if err := os.MkdirAll(n.logDir(), 0755); err != nil {
log.Fatal(context.Background(), err)
log.Fatalf(context.Background(), "%v", err)
}

joins := c.joins()
Expand Down Expand Up @@ -342,7 +342,7 @@ func (c *Cluster) isReplicated() (bool, string) {
if testutils.IsError(err, "(table|relation) \"crdb_internal.ranges\" does not exist") {
return true, ""
}
log.Fatal(context.Background(), err)
log.Fatalf(context.Background(), "%v", err)
}
defer rows.Close()

Expand Down Expand Up @@ -376,11 +376,11 @@ func (c *Cluster) UpdateZoneConfig(rangeMinBytes, rangeMaxBytes int64) {

buf, err := protoutil.Marshal(&zone)
if err != nil {
log.Fatal(context.Background(), err)
log.Fatalf(context.Background(), "%v", err)
}
_, err = c.Nodes[0].DB().Exec(`UPSERT INTO system.zones (id, config) VALUES (0, $1)`, buf)
if err != nil {
log.Fatal(context.Background(), err)
log.Fatalf(context.Background(), "%v", err)
}
}

Expand Down Expand Up @@ -506,7 +506,7 @@ func (n *Node) listeningURLFile() string {
// Start starts a node.
func (n *Node) Start(ctx context.Context, joins ...string) {
if err := <-n.StartAsync(ctx, joins...); err != nil {
log.Fatal(ctx, err)
log.Fatalf(ctx, "%v", err)
}
}

Expand Down Expand Up @@ -564,10 +564,10 @@ func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error

if err := n.cmd.Start(); err != nil {
if err := stdout.Close(); err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
}
if err := stderr.Close(); err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
}
return errors.Wrapf(err, "running %s %v", n.cmd.Path, n.cmd.Args)
}
Expand All @@ -577,13 +577,13 @@ func (n *Node) startAsyncInnerLocked(ctx context.Context, joins ...string) error
go func(cmd *exec.Cmd) {
waitErr := cmd.Wait()
if waitErr != nil {
log.Warning(ctx, waitErr)
log.Warningf(ctx, "%v", waitErr)
}
if err := stdout.Close(); err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
}
if err := stderr.Close(); err != nil {
log.Warning(ctx, err)
log.Warningf(ctx, "%v", err)
}

log.Infof(ctx, "process %d: %s", cmd.Process.Pid, cmd.ProcessState)
Expand Down Expand Up @@ -639,7 +639,7 @@ func portFromURL(rawURL string) (string, *url.URL, error) {
func makeDB(url string, numWorkers int, dbName string) *gosql.DB {
conn, err := gosql.Open("postgres", url)
if err != nil {
log.Fatal(context.Background(), err)
log.Fatalf(context.Background(), "%v", err)
}
if numWorkers == 0 {
numWorkers = 1
Expand Down Expand Up @@ -706,14 +706,14 @@ func (n *Node) waitUntilLive(dur time.Duration) error {

urlBytes, err := ioutil.ReadFile(n.listeningURLFile())
if err != nil {
log.Info(ctx, err)
log.Infof(ctx, "%v", err)
continue
}

var pgURL *url.URL
_, pgURL, err = portFromURL(string(urlBytes))
if err != nil {
log.Info(ctx, err)
log.Infof(ctx, "%v", err)
continue
}

Expand Down Expand Up @@ -749,13 +749,13 @@ func (n *Node) waitUntilLive(dur time.Duration) error {
if err := n.db.QueryRow(
`SELECT value FROM crdb_internal.node_runtime_info WHERE component='UI' AND field = 'URL'`,
).Scan(&uiStr); err != nil {
log.Info(ctx, err)
log.Infof(ctx, "%v", err)
return nil
}

_, uiURL, err = portFromURL(uiStr)
if err != nil {
log.Info(ctx, err)
log.Infof(ctx, "%v", err)
// TODO(tschottdorf): see above.
}
}
Expand Down Expand Up @@ -798,7 +798,7 @@ func (n *Node) Signal(s os.Signal) {
return
}
if err := n.cmd.Process.Signal(s); err != nil {
log.Warning(context.Background(), err)
log.Warningf(context.Background(), "%v", err)
}
}

Expand Down
10 changes: 5 additions & 5 deletions pkg/base/addr_validation.go
Original file line number Diff line number Diff line change
Expand Up @@ -286,7 +286,7 @@ func (cfg *Config) CheckCertificateAddrs(ctx context.Context) {
// with the provided certificate.
certInfo := cm.NodeCert()
if certInfo.Error != nil {
log.Shout(ctx, log.Severity_ERROR,
log.Shoutf(ctx, log.Severity_ERROR,
"invalid node certificate: %v", certInfo.Error)
} else {
cert := certInfo.ParsedCertificates[0]
Expand All @@ -313,11 +313,11 @@ func (cfg *Config) CheckCertificateAddrs(ctx context.Context) {
fmt.Fprintf(&msg, "advertise SQL address %q not in node certificate (%s)\n", host, addrInfo)
}
if msg.Len() > 0 {
log.Shout(ctx, log.Severity_WARNING,
fmt.Sprintf("%s"+
log.Shoutf(ctx, log.Severity_WARNING,
"%s"+
"Secure client connections are likely to fail.\n"+
"Consider extending the node certificate or tweak --listen-addr/--advertise-addr/--sql-addr/--advertise-sql-addr.",
msg.String()))
msg.String())
}
}

Expand All @@ -331,7 +331,7 @@ func (cfg *Config) CheckCertificateAddrs(ctx context.Context) {
certInfo = cm.NodeCert()
}
if certInfo.Error != nil {
log.Shout(ctx, log.Severity_ERROR,
log.Shoutf(ctx, log.Severity_ERROR,
"invalid UI certificate: %v", certInfo.Error)
} else {
cert := certInfo.ParsedCertificates[0]
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/changefeedccl/cdctest/nemeses.go
Original file line number Diff line number Diff line change
Expand Up @@ -637,7 +637,7 @@ func noteFeedMessage(a fsm.Args) error {
if err != nil {
return err
}
log.Info(a.Ctx, string(m.Resolved))
log.Infof(a.Ctx, "%v", string(m.Resolved))
return ns.v.NoteResolved(m.Partition, ts)
}
ts, _, err := ParseJSONValueTimestamps(m.Value)
Expand Down
4 changes: 2 additions & 2 deletions pkg/ccl/changefeedccl/sink.go
Original file line number Diff line number Diff line change
Expand Up @@ -271,13 +271,13 @@ type kafkaLogAdapter struct {
var _ sarama.StdLogger = (*kafkaLogAdapter)(nil)

func (l *kafkaLogAdapter) Print(v ...interface{}) {
log.InfoDepth(l.ctx, 1, v...)
log.InfofDepth(l.ctx, 1, "", v...)
}
func (l *kafkaLogAdapter) Printf(format string, v ...interface{}) {
log.InfofDepth(l.ctx, 1, format, v...)
}
func (l *kafkaLogAdapter) Println(v ...interface{}) {
log.InfoDepth(l.ctx, 1, v...)
log.InfofDepth(l.ctx, 1, "", v...)
}

func init() {
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/load.go
Original file line number Diff line number Diff line change
Expand Up @@ -384,7 +384,7 @@ func writeSST(
}

filename := fmt.Sprintf("load-%d.sst", rand.Int63())
log.Info(ctx, "writesst ", filename)
log.Infof(ctx, "writesst %s", filename)

sstFile := &storage.MemFile{}
sst := storage.MakeBackupSSTWriter(sstFile)
Expand Down
2 changes: 1 addition & 1 deletion pkg/ccl/importccl/read_import_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -393,7 +393,7 @@ type importFileContext struct {
// handleCorruptRow reports an error encountered while processing a row
// in an input file.
func handleCorruptRow(ctx context.Context, fileCtx *importFileContext, err error) error {
log.Error(ctx, err)
log.Errorf(ctx, "%v", err)

if rowErr, isRowErr := err.(*importRowError); isRowErr && fileCtx.rejected != nil {
fileCtx.rejected <- rowErr.row + "\n"
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/debug.go
Original file line number Diff line number Diff line change
Expand Up @@ -863,7 +863,7 @@ func runTimeSeriesDump(cmd *cobra.Command, args []string) error {
tsClient := tspb.NewTimeSeriesClient(conn)
stream, err := tsClient.Dump(context.Background(), &tspb.DumpRequest{})
if err != nil {
log.Fatal(context.Background(), err)
log.Fatalf(context.Background(), "%v", err)
}

var name, source string
Expand Down
11 changes: 5 additions & 6 deletions pkg/cli/demo_cluster.go
Original file line number Diff line number Diff line change
Expand Up @@ -433,18 +433,17 @@ func maybeWarnMemSize(ctx context.Context) {
requestedMem := (demoCtx.cacheSize + demoCtx.sqlPoolMemorySize) * int64(demoCtx.nodes)
maxRecommendedMem := int64(.75 * float64(maxMemory))
if requestedMem > maxRecommendedMem {
log.Shout(
log.Shoutf(
ctx,
log.Severity_WARNING,
fmt.Sprintf(`HIGH MEMORY USAGE
`HIGH MEMORY USAGE
The sum of --max-sql-memory (%s) and --cache (%s) multiplied by the
number of nodes (%d) results in potentially high memory usage on your
device.
This server is running at increased risk of memory-related failures.`,
demoNodeSQLMemSizeValue,
demoNodeCacheSizeValue,
demoCtx.nodes,
),
demoNodeSQLMemSizeValue,
demoNodeCacheSizeValue,
demoCtx.nodes,
)
}
}
Expand Down
6 changes: 3 additions & 3 deletions pkg/cli/error.go
Original file line number Diff line number Diff line change
Expand Up @@ -354,11 +354,11 @@ func maybeShoutError(
}

func checkAndMaybeShout(err error) error {
return checkAndMaybeShoutTo(err, log.Shout)
return checkAndMaybeShoutTo(err, log.Shoutf)
}

func checkAndMaybeShoutTo(
err error, logger func(context.Context, log.Severity, ...interface{}),
err error, logger func(context.Context, log.Severity, string, ...interface{}),
) error {
if err == nil {
return nil
Expand All @@ -370,6 +370,6 @@ func checkAndMaybeShoutTo(
severity = ec.severity
cause = ec.cause
}
logger(context.Background(), severity, cause)
logger(context.Background(), severity, "%v", cause)
return err
}
2 changes: 1 addition & 1 deletion pkg/cli/error_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -109,7 +109,7 @@ type logger struct {
Err error
}

func (l *logger) Log(_ context.Context, sev log.Severity, args ...interface{}) {
func (l *logger) Log(_ context.Context, sev log.Severity, msg string, args ...interface{}) {
require.Equal(l.TB, 1, len(args), "expected to log one item")
err, ok := args[0].(error)
require.True(l.TB, ok, "expected to log an error")
Expand Down
2 changes: 1 addition & 1 deletion pkg/cli/sql_util.go
Original file line number Diff line number Diff line change
Expand Up @@ -448,7 +448,7 @@ func (c *sqlConn) Close() {
if c.conn != nil {
err := c.conn.Close()
if err != nil && err != driver.ErrBadConn {
log.Info(context.TODO(), err)
log.Infof(context.TODO(), "%v", err)
}
c.conn = nil
}
Expand Down
Loading

0 comments on commit aebcb79

Please sign in to comment.