Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
71140: sql: NFC-normalization on double-quoted identifiers r=rafiss a=e-mbrown

Resolves: #55396

There was a lack of NFC-normalization on double-quoted
identifiers. This allowed for ambiguous table/db/column names.
This change adds normalization and prevents this case.

Release note: None

71175: ui/cluster-ui: make app name a query search parameter in stmts page r=maryliag,matthewtodd a=xinhaoz

Fixes: #70790

Previously, the selected app was derived from a route param on the
statements page. All other filters are derived from query search
parameters on the page. This commit makes the app name a query search
parameter, as is the case in the transactions page.

Release note: None

71405: vendor: bump Pebble to 59007c613a66 r=jbowens a=nicktrav

```
59007c61 sstable: consolidate checksum types
17635b0a *: address staticcheck lints
11823273 *: address staticcheck lint check U1000
807abfe8 *: address staticcheck lint check S1039
d24dd342 all: remove some unused code
3bdca93a sstable: clarify comment on checksum input
926d23e9 pebble: remove comment related to batching from the table cache
0a6177ae db: tweak ErrClosed documentation
ecc685b2 db: expose facility for retrieving batch count
b2eb88a7 sstable: remove unused rawBlockIter err field
```

Release note: None.

71419: sqlliveness: session expiry callbacks must be async r=ajwerner,jaylim-crl a=dhartunian

Previously, the sqlliveness session expiry callbacks were called in the
heartbeatLoop thread which executed the renew/expiry logic. This could
cause deadlock since session expiration is used to trigger a shutdown of
the SQL instance via `stopper.Stop()`. The stopper would wait for all
async tasks to quiesce, but the `heartbeatLoop` would continue, waiting
for the callbacks to finish running. In addition, this task would hold a
lock on `l.mu` while waiting for the callbacks to run causing other
threads to wait if they needed to retrieve the session.

This change invokes each callback in its own goroutine to prevent this
deadlock.

Resolves #71292

Release note: None

Co-authored-by: e-mbrown <ebsonari@gmail.com>
Co-authored-by: Xin Hao Zhang <xzhang@cockroachlabs.com>
Co-authored-by: Nick Travers <travers@cockroachlabs.com>
Co-authored-by: David Hartunian <davidh@cockroachlabs.com>
  • Loading branch information
5 people committed Oct 12, 2021
5 parents ecb3981 + 808ef25 + 786f694 + 8bb62b2 + 3586291 commit 6159a73
Show file tree
Hide file tree
Showing 23 changed files with 212 additions and 76 deletions.
4 changes: 2 additions & 2 deletions DEPS.bzl
Original file line number Diff line number Diff line change
Expand Up @@ -736,8 +736,8 @@ def go_deps():
name = "com_github_cockroachdb_pebble",
build_file_proto_mode = "disable_global",
importpath = "github.com/cockroachdb/pebble",
sum = "h1:6Ur6xotJWd8Z1r6zsyI9vHVp+pvA/1dFMpbX+avdjxk=",
version = "v0.0.0-20210930201120-c73841491dd5",
sum = "h1:d+OlrpxjR43JYIIdHzADu7CbsfBxsufssvpFiRUdAxg=",
version = "v0.0.0-20211011160653-59007c613a66",
)

go_repository(
Expand Down
2 changes: 1 addition & 1 deletion go.mod
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ require (
github.com/cockroachdb/go-test-teamcity v0.0.0-20191211140407-cff980ad0a55
github.com/cockroachdb/gostdlib v1.13.0
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f
github.com/cockroachdb/pebble v0.0.0-20210930201120-c73841491dd5
github.com/cockroachdb/pebble v0.0.0-20211011160653-59007c613a66
github.com/cockroachdb/redact v1.1.3
github.com/cockroachdb/returncheck v0.0.0-20200612231554-92cdbca611dd
github.com/cockroachdb/sentry-go v0.6.1-cockroachdb.2
Expand Down
4 changes: 2 additions & 2 deletions go.sum
Original file line number Diff line number Diff line change
Expand Up @@ -302,8 +302,8 @@ github.com/cockroachdb/gostdlib v1.13.0 h1:TzSEPYgkKDNei3gbLc0rrHu4iHyBp7/+NxPOF
github.com/cockroachdb/gostdlib v1.13.0/go.mod h1:eXX95p9QDrYwJfJ6AgeN9QnRa/lqqid9LAzWz/l5OgA=
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f h1:o/kfcElHqOiXqcou5a3rIlMc7oJbMQkeLk0VQJ7zgqY=
github.com/cockroachdb/logtags v0.0.0-20190617123548-eb05cc24525f/go.mod h1:i/u985jwjWRlyHXQbwatDASoW0RMlZ/3i9yJHE2xLkI=
github.com/cockroachdb/pebble v0.0.0-20210930201120-c73841491dd5 h1:6Ur6xotJWd8Z1r6zsyI9vHVp+pvA/1dFMpbX+avdjxk=
github.com/cockroachdb/pebble v0.0.0-20210930201120-c73841491dd5/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU=
github.com/cockroachdb/pebble v0.0.0-20211011160653-59007c613a66 h1:d+OlrpxjR43JYIIdHzADu7CbsfBxsufssvpFiRUdAxg=
github.com/cockroachdb/pebble v0.0.0-20211011160653-59007c613a66/go.mod h1:buxOO9GBtOcq1DiXDpIPYrmxY020K2A8lOrwno5FetU=
github.com/cockroachdb/redact v1.0.8/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.1.0/go.mod h1:BVNblN9mBWFyMyqK1k3AAiSxhvhfK2oOZZ2lK+dpvRg=
github.com/cockroachdb/redact v1.1.3 h1:AKZds10rFSIj7qADf0g46UixK8NNLwWTNdCIGS5wfSQ=
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,6 @@ p CREATE TABLE public.p (
) PARTITION BY LIST (a) (
PARTITION p1 VALUES IN ((1)),
PARTITION "P1" VALUES IN ((2)),
PARTITION "Amélie" VALUES IN ((3))
PARTITION "Amélie" VALUES IN ((3))
)
-- Warning: Partitioned table with no zone configurations.
1 change: 1 addition & 0 deletions pkg/sql/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -478,6 +478,7 @@ go_test(
"metric_test.go",
"metric_util_test.go",
"mutation_test.go",
"normalization_test.go",
"partition_test.go",
"pg_metadata_test.go",
"pg_oid_test.go",
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/lexbase/normalize.go
Original file line number Diff line number Diff line change
Expand Up @@ -53,3 +53,12 @@ func NormalizeName(n string) string {
}
return norm.NFC.String(lower)
}

// NormalizeString normalizes to Unicode Normalization Form C (NFC).
// This function is specifically for double quoted identifiers.
func NormalizeString(s string) string {
if isASCII(s) {
return s
}
return norm.NFC.String(s)
}
22 changes: 2 additions & 20 deletions pkg/sql/logictest/testdata/logic_test/case_sensitive_names
Original file line number Diff line number Diff line change
Expand Up @@ -251,28 +251,10 @@ DROP INDEX a@I

# Unicode sequences are preserved.

statement ok
# Check that normalization occurs
statement error duplicate column name: "Amélie"
CREATE TABLE Amelie("Amélie" INT, "Amélie" INT); INSERT INTO Amelie VALUES (1, 2)

# Check that the column names were encoded properly
query I
SELECT ordinal_position FROM information_schema.columns WHERE table_name = 'amelie' AND column_name::BYTES = b'Ame\xcc\x81lie'
----
1

query I
SELECT ordinal_position FROM information_schema.columns WHERE table_name = 'amelie' AND column_name::BYTES = b'Am\xc3\xa9lie'
----
2

# Check that the non-normalized names propagate throughout until results.

query II colnames
SELECT "Amélie", "Amélie" FROM Amelie
----
Amélie Amélie
2 1

# Check that function names are also recognized case-insensitively.
query I
SELECT LENGTH('abc') -- lint: uppercase function OK
Expand Down
63 changes: 63 additions & 0 deletions pkg/sql/normalization_test.go
Original file line number Diff line number Diff line change
@@ -0,0 +1,63 @@
// Copyright 2021 The Cockroach Authors.
//
// Use of this software is governed by the Business Source License
// included in the file licenses/BSL.txt.
//
// As of the Change Date specified in that file, in accordance with
// the Business Source License, use of this software will be governed
// by the Apache License, Version 2.0, included in the file
// licenses/APL.txt.

package sql_test

import (
"context"
"strings"
"testing"

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
)

func TestNFCNormalization(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
s, db, _ := serverutils.StartServer(t, base.TestServerArgs{Insecure: true})
defer s.Stopper().Stop(context.Background())
defer db.Close()

_, err := db.Exec("CREATE TABLE café (a INT)")
require.NoError(t, err)

_, err = db.Exec("CREATE TABLE Cafe\u0301 (a INT)")
require.Errorf(t, err, "The tables should be considered duplicates when normalized")
require.True(t, strings.Contains(err.Error(), "already exists"))

_, err = db.Exec("CREATE TABLE cafe\u0301 (a INT)")
require.Errorf(t, err, "The tables should be considered duplicates when normalized")
require.True(t, strings.Contains(err.Error(), "already exists"))

_, err = db.Exec("CREATE TABLE caf\u00E9 (a INT)")
require.Errorf(t, err, "The tables should be considered duplicates when normalized")
require.True(t, strings.Contains(err.Error(), "already exists"))

_, err = db.Exec("CREATE TABLE \"caf\u00E9\" (a INT)")
require.Errorf(t, err, "The tables should be considered duplicates when normalized")
require.True(t, strings.Contains(err.Error(), "already exists"))

_, err = db.Exec("CREATE TABLE \"cafe\u0301\" (a INT)")
require.Errorf(t, err, "The tables should be considered duplicates when normalized")
require.True(t, strings.Contains(err.Error(), "already exists"))

_, err = db.Exec(`CREATE TABLE "Café" (a INT)`)
require.NoError(t, err)
//Ensure normal strings are not normalized like double quoted strings
var b bool
err = db.QueryRow("SELECT 'caf\u00E9' = 'cafe\u0301'").Scan(&b)
require.NoError(t, err)
require.False(t, b)

}
9 changes: 6 additions & 3 deletions pkg/sql/scanner/scan.go
Original file line number Diff line number Diff line change
Expand Up @@ -563,7 +563,7 @@ func (s *Scanner) scanIdent(lval ScanSymType) {
}
//fmt.Println("parsed: ", s.in[start:s.pos], isASCII, isLower)

if isLower {
if isLower && isASCII {
// Already lowercased - nothing to do.
lval.SetStr(s.in[start:s.pos])
} else if isASCII {
Expand Down Expand Up @@ -830,7 +830,6 @@ func (s *Scanner) scanString(lval ScanSymType, ch int, allowEscapes, requireUTF8
buf := s.buffer()
var runeTmp [utf8.UTFMax]byte
start := s.pos

outer:
for {
switch s.next() {
Expand Down Expand Up @@ -918,7 +917,11 @@ outer:
return false
}

lval.SetStr(s.finishString(buf))
if ch == identQuote {
lval.SetStr(lexbase.NormalizeString(s.finishString(buf)))
} else {
lval.SetStr(s.finishString(buf))
}
return true
}

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/sqlliveness/slinstance/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -32,6 +32,7 @@ go_test(
"//pkg/settings/cluster",
"//pkg/sql/sqlliveness",
"//pkg/sql/sqlliveness/slstorage",
"//pkg/testutils",
"//pkg/util/hlc",
"//pkg/util/leaktest",
"//pkg/util/log",
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/sqlliveness/slinstance/slinstance.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,9 @@ func (s *session) Expiration() hlc.Timestamp {
return s.mu.exp
}

// RegisterCallbackForSessionExpiry adds the given function to the list
// of functions called after a session expires. The functions are
// executed in a goroutine.
func (s *session) RegisterCallbackForSessionExpiry(sExp func(context.Context)) {
s.mu.Lock()
defer s.mu.Unlock()
Expand All @@ -86,7 +89,7 @@ func (s *session) invokeSessionExpiryCallbacks(ctx context.Context) {
s.mu.RLock()
defer s.mu.RUnlock()
for _, callback := range s.mu.sessionExpiryCallbacks {
callback(ctx)
go callback(ctx)
}
}

Expand Down
43 changes: 43 additions & 0 deletions pkg/sql/sqlliveness/slinstance/slinstance_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package slinstance_test

import (
"context"
"errors"
"testing"
"time"

Expand All @@ -20,13 +21,55 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slinstance"
"github.com/cockroachdb/cockroach/pkg/sql/sqlliveness/slstorage"
"github.com/cockroachdb/cockroach/pkg/testutils"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/cockroachdb/cockroach/pkg/util/stop"
"github.com/stretchr/testify/require"
)

func TestSQLInstance_invokesSessionExpiryCallbacksInGoroutine(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)

ctx, stopper := context.Background(), stop.NewStopper()

t0 := time.Date(2000, time.January, 1, 0, 0, 0, 0, time.UTC)
mClock := hlc.NewManualClock(t0.UnixNano())
clock := hlc.NewClock(mClock.UnixNano, time.Nanosecond)
settings := cluster.MakeTestingClusterSettings()

fakeStorage := slstorage.NewFakeStorage()
sqlInstance := slinstance.NewSQLInstance(stopper, clock, fakeStorage, settings, nil)
sqlInstance.Start(ctx)

session, err := sqlInstance.Session(ctx)
require.NoError(t, err)

// Simulating what happens in `instanceprovider.shutdownSQLInstance`
session.RegisterCallbackForSessionExpiry(func(ctx context.Context) {
stopper.Stop(ctx)
})

// Removing the session will run the callback above, which will have to
// wait for async tasks to stop. The async tasks include the
// sqlInstance `heartbeatLoop` function.
require.NoError(t, fakeStorage.Delete(ctx, session.ID()))

// Clock needs to advance for expiry we trigger below to be valid
mClock.Increment(int64(slinstance.DefaultTTL.Get(&settings.SV)))

testutils.SucceedsSoon(t, func() error {
select {
case <-stopper.IsStopped():
return nil
default:
return errors.New("not stopped")
}
})
}

func TestSQLInstance(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
8 changes: 6 additions & 2 deletions pkg/storage/batch.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package storage

import (
"encoding/binary"
"math"

"github.com/cockroachdb/cockroach/pkg/storage/enginepb"
"github.com/cockroachdb/cockroach/pkg/util/hlc"
Expand Down Expand Up @@ -122,8 +123,11 @@ func rocksDBBatchDecodeHeader(repr []byte) (count int, orepr pebble.BatchReader,
if seq != 0 {
return 0, nil, errors.Errorf("bad sequence: expected 0, but found %d", seq)
}
count = int(binary.LittleEndian.Uint32(repr[countPos:headerSize]))
return count, pebble.MakeBatchReader(repr), nil
r, c := pebble.ReadBatch(repr)
if c > math.MaxInt32 {
return 0, nil, errors.Errorf("count %d would overflow max int", c)
}
return int(c), r, nil
}

// RocksDBBatchReader is used to iterate the entries in a RocksDB batch
Expand Down
30 changes: 22 additions & 8 deletions pkg/ui/workspaces/cluster-ui/src/queryFilter/filter.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ const timeUnit = [
];

export const defaultFilters: Filters = {
app: "All",
app: "",
timeNumber: "0",
timeUnit: "seconds",
fullScan: false,
Expand All @@ -89,15 +89,17 @@ export const defaultFilters: Filters = {
* @return Filters: the default filters with updated keys existing on
* queryString
*/
export const getFiltersFromQueryString = (queryString: string) => {
export const getFiltersFromQueryString = (
queryString: string,
): Record<string, string> => {
const searchParams = new URLSearchParams(queryString);

return Object.keys(defaultFilters).reduce(
(filters, filter: keyof Filters): Filters => {
const defaultValue = defaultFilters[filter];
const queryStringFilter = searchParams.get(filter);
const filterValue =
queryStringFilter === null
queryStringFilter == null
? defaultValue
: defaultValue.constructor(searchParams.get(filter));
return { [filter]: filterValue, ...filters };
Expand All @@ -114,7 +116,7 @@ export const getFiltersFromQueryString = (queryString: string) => {
* we want to consider 0 active Filters
*/
export const inactiveFiltersState: Filters = {
app: "All",
app: "",
timeNumber: "0",
fullScan: false,
sqlType: "",
Expand All @@ -123,7 +125,7 @@ export const inactiveFiltersState: Filters = {
nodes: "",
};

export const calculateActiveFilters = (filters: Filters) => {
export const calculateActiveFilters = (filters: Filters): number => {
return Object.keys(inactiveFiltersState).reduce(
(active, filter: keyof Filters) => {
return inactiveFiltersState[filter] !== filters[filter]
Expand Down Expand Up @@ -185,7 +187,19 @@ export class Filter extends React.Component<QueryFilter, FilterState> {
this.setState({ hide: true });
};

handleChange = (event: any, field: string) => {
handleSelectChange = (
event: { label: string; value: string },
field: string,
): void => {
this.setState({
filters: {
...this.state.filters,
[field]: event.value,
},
});
};

handleChange = (event: any, field: string): void => {
this.setState({
filters: {
...this.state.filters,
Expand Down Expand Up @@ -419,7 +433,7 @@ export class Filter extends React.Component<QueryFilter, FilterState> {
<div className={filterLabel.top}>App</div>
<Select
options={apps}
onChange={e => this.handleChange(e, "app")}
onChange={e => this.handleSelectChange(e, "app")}
value={apps.filter(app => app.value === filters.app)}
placeholder="All"
styles={customStyles}
Expand All @@ -441,7 +455,7 @@ export class Filter extends React.Component<QueryFilter, FilterState> {
<Select
options={timeUnit}
value={timeUnit.filter(unit => unit.label == filters.timeUnit)}
onChange={e => this.handleChange(e, "timeUnit")}
onChange={e => this.handleSelectChange(e, "timeUnit")}
className={timePair.timeUnit}
styles={customStylesSmall}
/>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -150,7 +150,11 @@ export const selectStatement = createSelector(
statement,
stats: combineStatementStats(results.map(s => s.stats)),
byNode: coalesceNodeStats(results),
app: _.uniq(results.map(s => s.app)),
app: _.uniq(
results.map(s =>
s.app.startsWith(internalAppNamePrefix) ? "(internal)" : s.app,
),
),
database: queryByName(props.location, databaseAttr),
distSQL: fractionMatching(results, s => s.distSQL),
vec: fractionMatching(results, s => s.vec),
Expand Down
Loading

0 comments on commit 6159a73

Please sign in to comment.