Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
75447: storage: fix `EngineComparer` ordering of bare suffixes r=jbowens a=erikgrinaker

For generalized range key support, Pebble now requires bare key suffixes
to be comparable with the same ordering as if they had a common user key
prefix (see [cockroachdb/pebble#08c5d46c75722b9527c360ca1e7069c0d2286e59](cockroachdb/pebble@08c5d46)).

This patch modifies `EngineComparer` to satisfy that requirement.
Previously, such suffixes would sort in ascending timestamp order
rather than the correct descending order.

Release note: None

75462: pgwire: fix a flake in TestAuthenticationAndHBARules r=cameronnunez a=knz

Fixes #75286.

The regexp was sometimes capturing the conn_end event from the
previous test directive.

Release note: None

75470: ui: remove `$ internal` from apps filter option r=maryliag a=maryliag

Previously, we had the option `$ internal` on the list of
Apps on filters of both Statement and Transaction pages.
We're making a change to not return internal filters anymore,
so this commit remove that option from the filter, since
it will always results on a blank table.

Release note (ui change): Remove `$ internal` as one of the apps
option under the Statements and Transactions page filters.

75473: ui: fix 3rd click on sorting table column r=maryliag a=maryliag

Previously, when a user clicked the 3rd time on the same
column, the page was crashing (Only happening on
Statement and Transaction pages). This would happen
because on those two pages we need to update when we switch
tabs, and the value of sort being set to null was
causing the componentDidUpdate being called constantly.
Since we always have a sort happening on the table, this
commit removes the 3rd option of blank selection.
This commit also updates the Session tab to be properly
updating the query params when the tab is switched.

Fixes #74440

Release note (bug fix): Fix SQL Activity pages crashing when
a column was sorted by the 3rd time.

75476: sql: remove KeyToDescTranslator r=RaduBerinde a=RaduBerinde

This interface was only useful with interleaved tables.

Release note: None

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Raphael 'kena' Poss <[email protected]>
Co-authored-by: Marylia Gutierrez <[email protected]>
Co-authored-by: Radu Berinde <[email protected]>
  • Loading branch information
5 people committed Jan 25, 2022
6 parents 44b2ba8 + 1b935f9 + d226abe + 997cbfb + 6b64dfe + ac48c19 commit 1c4b0dd
Show file tree
Hide file tree
Showing 12 changed files with 105 additions and 122 deletions.
22 changes: 1 addition & 21 deletions pkg/sql/colfetcher/cfetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -1435,31 +1435,11 @@ func (rf *cFetcher) getCurrentColumnFamilyID() (descpb.FamilyID, error) {
// error may also undergo a mapping to make it more user friendly for SQL
// consumers.
func (rf *cFetcher) convertFetchError(ctx context.Context, err error) error {
err = row.ConvertFetchError(ctx, rf, err)
err = row.ConvertFetchError(ctx, rf.table.desc, err)
err = colexecerror.NewStorageError(err)
return err
}

// KeyToDesc implements the KeyToDescTranslator interface. The implementation is
// used by convertFetchError.
func (rf *cFetcher) KeyToDesc(key roachpb.Key) (catalog.TableDescriptor, bool) {
if len(key) < rf.table.knownPrefixLength {
return nil, false
}
nIndexCols := rf.table.index.NumKeyColumns() + rf.table.index.NumKeySuffixColumns()
tableKeyVals := make([]rowenc.EncDatum, nIndexCols)
_, _, err := rowenc.DecodeKeyVals(
rf.table.keyValTypes,
tableKeyVals,
rf.table.indexColumnDirs,
key[rf.table.knownPrefixLength:],
)
if err != nil {
return nil, false
}
return rf.table.desc, true
}

// getBytesRead returns the number of bytes read by the cFetcher throughout its
// existence so far. This number accumulates the bytes read statistic across
// StartScan* and Close methods.
Expand Down
3 changes: 1 addition & 2 deletions pkg/sql/pgwire/testdata/auth/scram
Original file line number Diff line number Diff line change
Expand Up @@ -234,10 +234,9 @@ connect user=foo password=abc
ok defaultdb

# Assert the conn used a cleartext handshake.
authlog 4
authlog 3
.*client_connection_end
----
72 {"EventType":"client_authentication_info","Info":"found stored crdb-bcrypt credentials; requesting cleartext password","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","SystemIdentity":"foo","Timestamp":"XXX","Transport":"hostssl","User":"foo"}
73 {"EventType":"client_authentication_ok","InstanceID":1,"Method":"cert-password","Network":"tcp","RemoteAddress":"XXX","SystemIdentity":"foo","Timestamp":"XXX","Transport":"hostssl","User":"foo"}
74 {"Duration":"NNN","EventType":"client_session_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"}
75 {"Duration":"NNN","EventType":"client_connection_end","InstanceID":1,"Network":"tcp","RemoteAddress":"XXX","Timestamp":"XXX"}
Expand Down
11 changes: 1 addition & 10 deletions pkg/sql/row/errors.go
Original file line number Diff line number Diff line change
Expand Up @@ -76,25 +76,16 @@ func ConvertBatchError(ctx context.Context, tableDesc catalog.TableDescriptor, b
return origPErr.GoError()
}

// KeyToDescTranslator is capable of translating a key found in an error to a
// table descriptor for error reporting.
type KeyToDescTranslator interface {
// KeyToDesc attempts to translate the key found in an error to a table
// descriptor.
KeyToDesc(roachpb.Key) (catalog.TableDescriptor, bool)
}

// ConvertFetchError attempts to map a key-value error generated during a
// key-value fetch to a user friendly SQL error.
func ConvertFetchError(ctx context.Context, descForKey KeyToDescTranslator, err error) error {
func ConvertFetchError(ctx context.Context, desc catalog.TableDescriptor, err error) error {
var errs struct {
wi *roachpb.WriteIntentError
bs *roachpb.MinTimestampBoundUnsatisfiableError
}
switch {
case errors.As(err, &errs.wi):
key := errs.wi.Intents[0].Key
desc, _ := descForKey.KeyToDesc(key)
return NewLockNotAvailableError(ctx, desc, key, errs.wi.Reason)
case errors.As(err, &errs.bs):
return pgerror.WithCandidateCode(
Expand Down
14 changes: 1 addition & 13 deletions pkg/sql/row/fetcher.go
Original file line number Diff line number Diff line change
Expand Up @@ -626,7 +626,7 @@ func (rf *Fetcher) setNextKV(kv roachpb.KeyValue, needsCopy bool) {
func (rf *Fetcher) nextKey(ctx context.Context) (newRow bool, _ error) {
ok, kv, finalReferenceToBatch, err := rf.kvFetcher.NextKV(ctx, rf.mvccDecodeStrategy)
if err != nil {
return false, ConvertFetchError(ctx, rf, err)
return false, ConvertFetchError(ctx, rf.table.desc, err)
}
rf.setNextKV(kv, finalReferenceToBatch)

Expand Down Expand Up @@ -720,18 +720,6 @@ func (rf *Fetcher) DecodeIndexKey(key roachpb.Key) (remaining []byte, foundNull
)
}

// KeyToDesc implements the KeyToDescTranslator interface. The implementation is
// used by ConvertFetchError.
func (rf *Fetcher) KeyToDesc(key roachpb.Key) (catalog.TableDescriptor, bool) {
if len(key) < rf.table.knownPrefixLength {
return nil, false
}
if _, _, err := rf.DecodeIndexKey(key); err != nil {
return nil, false
}
return rf.table.desc, true
}

// processKV processes the given key/value, setting values in the row
// accordingly. If debugStrings is true, returns pretty printed key and value
// information in prettyKey/prettyValue (otherwise they are empty strings).
Expand Down
8 changes: 7 additions & 1 deletion pkg/storage/pebble.go
Original file line number Diff line number Diff line change
Expand Up @@ -93,9 +93,15 @@ func EngineKeyCompare(a, b []byte) int {
return bytes.Compare(a, b)
}

// Compute the index of the separator between the key and the version.
// Compute the index of the separator between the key and the version. If the
// separator is found to be at -1 for both keys, then we are comparing bare
// suffixes without a user key part. Pebble requires bare suffixes to be
// comparable with the same ordering as if they had a common user key.
aSep := aEnd - int(a[aEnd])
bSep := bEnd - int(b[bEnd])
if aSep == -1 && bSep == -1 {
aSep, bSep = 0, 0 // comparing bare suffixes
}
if aSep < 0 || bSep < 0 {
// This should never happen unless there is some sort of corruption of
// the keys.
Expand Down
34 changes: 34 additions & 0 deletions pkg/storage/pebble_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -44,6 +44,40 @@ import (
"github.com/stretchr/testify/require"
)

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

keyAMetadata := MVCCKey{
Key: []byte("a"),
}
keyA2 := MVCCKey{
Key: []byte("a"),
Timestamp: hlc.Timestamp{WallTime: 2},
}
keyA1 := MVCCKey{
Key: []byte("a"),
Timestamp: hlc.Timestamp{WallTime: 1},
}
keyB2 := MVCCKey{
Key: []byte("b"),
Timestamp: hlc.Timestamp{WallTime: 2},
}

require.Equal(t, -1, EngineComparer.Compare(EncodeKey(keyAMetadata), EncodeKey(keyA1)),
"expected key metadata to sort first")
require.Equal(t, -1, EngineComparer.Compare(EncodeKey(keyA2), EncodeKey(keyA1)),
"expected higher timestamp to sort first")
require.Equal(t, -1, EngineComparer.Compare(EncodeKey(keyA2), EncodeKey(keyB2)),
"expected lower key to sort first")

suffix := func(key []byte) []byte {
return key[EngineComparer.Split(key):]
}
require.Equal(t, -1, EngineComparer.Compare(suffix(EncodeKey(keyA2)), suffix(EncodeKey(keyA1))),
"expected bare suffix with higher timestamp to sort first")
}

func TestPebbleTimeBoundPropCollector(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
Expand Down
31 changes: 0 additions & 31 deletions pkg/storage/sst_iterator_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -126,34 +126,3 @@ func TestSSTIterator(t *testing.T) {
runTestSSTIterator(t, iter, allKVs)
})
}

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

keyAMetadata := MVCCKey{
Key: []byte("a"),
}
keyA2 := MVCCKey{
Key: []byte("a"),
Timestamp: hlc.Timestamp{WallTime: 2},
}
keyA1 := MVCCKey{
Key: []byte("a"),
Timestamp: hlc.Timestamp{WallTime: 1},
}
keyB2 := MVCCKey{
Key: []byte("b"),
Timestamp: hlc.Timestamp{WallTime: 2},
}

if x := EngineComparer.Compare(EncodeKey(keyAMetadata), EncodeKey(keyA1)); x != -1 {
t.Errorf("expected key metadata to sort first got: %d", x)
}
if x := EngineComparer.Compare(EncodeKey(keyA2), EncodeKey(keyA1)); x != -1 {
t.Errorf("expected higher timestamp to sort first got: %d", x)
}
if x := EngineComparer.Compare(EncodeKey(keyA2), EncodeKey(keyB2)); x != -1 {
t.Errorf("expected lower key to sort first got: %d", x)
}
}
17 changes: 16 additions & 1 deletion pkg/ui/workspaces/cluster-ui/src/sessions/sessionsPage.tsx
Original file line number Diff line number Diff line change
Expand Up @@ -26,7 +26,11 @@ import emptyTableResultsIcon from "../assets/emptyState/empty-table-results.svg"
import SQLActivityError from "../sqlActivity/errorComponent";

import { Pagination, ResultsPerPageLabel } from "src/pagination";
import { SortSetting, ISortedTablePagination } from "src/sortedtable";
import {
SortSetting,
ISortedTablePagination,
updateSortSettingQueryParamsOnTab,
} from "src/sortedtable";
import { Loading } from "src/loading";
import { Anchor } from "src/anchor";
import { EmptyTable } from "src/empty";
Expand Down Expand Up @@ -137,7 +141,18 @@ export class SessionsPage extends React.Component<
}

componentDidUpdate = (): void => {
const { history, sortSetting } = this.props;

this.props.refreshSessions();
updateSortSettingQueryParamsOnTab(
"Sessions",
sortSetting,
{
ascending: false,
columnTitle: "statementAge",
},
history,
);
};

onChangePage = (current: number): void => {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -35,20 +35,18 @@ export const TableHead: React.FC<TableHeadProps> = ({
const cellContentWrapper = cx("inner-content-wrapper");
const arrowsClass = cx("sortable__actions");

function handleSort(picked: boolean, columnTitle: string) {
function handleSort(
newColumnSelected: boolean,
columnTitle: string,
prevValue: boolean,
) {
// If the columnTitle is different than the previous value, initial sort
// descending. If the same columnTitle is clicked multiple times consecutively,
// first change to ascending, then remove the sort key.
const ASCENDING = true;
const DESCENDING = false;

const direction = picked ? ASCENDING : DESCENDING;
const sortElementColumnTitle =
picked && sortSetting.ascending ? null : columnTitle;
// descending. If is the same columnTitle the value is updated.

const ascending = newColumnSelected ? false : !prevValue;
onChangeSortSetting({
ascending: direction,
columnTitle: sortElementColumnTitle,
ascending,
columnTitle,
});
}

Expand All @@ -58,15 +56,21 @@ export const TableHead: React.FC<TableHeadProps> = ({
{expandableConfig && <th className={thClass} />}
{columns.map((c: SortableColumn, idx: number) => {
const sortable = c.columnTitle !== (null || undefined);
const picked = c.name === sortSetting.columnTitle;
const newColumnSelected = c.name !== sortSetting.columnTitle;
const style = { textAlign: c.titleAlign };
const cellAction = sortable ? () => handleSort(picked, c.name) : null;
const cellAction = sortable
? () => handleSort(newColumnSelected, c.name, sortSetting.ascending)
: null;
const cellClasses = cx(
"head-wrapper__cell",
"sorted__cell",
sortable && "sorted__cell--sortable",
sortSetting.ascending && picked && "sorted__cell--ascending",
!sortSetting.ascending && picked && "sorted__cell--descending",
sortSetting.ascending &&
!newColumnSelected &&
"sorted__cell--ascending",
!sortSetting.ascending &&
!newColumnSelected &&
"sorted__cell--descending",
firstCellBordered && idx === 0 && "cell-header",
);
const titleClasses = c.hideTitleUnderline ? "" : cx("column-title");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -61,28 +61,27 @@ export const selectApps = createSelector(sqlStatsSelector, sqlStatsState => {
}

let sawBlank = false;
let sawInternal = false;
const apps: { [app: string]: boolean } = {};
sqlStatsState.data.statements.forEach(
(statement: ICollectedStatementStatistics) => {
if (
const isNotInternalApp =
sqlStatsState.data.internal_app_name_prefix &&
statement.key.key_data.app.startsWith(
!statement.key.key_data.app.startsWith(
sqlStatsState.data.internal_app_name_prefix,
)
);
if (
sqlStatsState.data.internal_app_name_prefix == undefined ||
isNotInternalApp
) {
sawInternal = true;
} else if (statement.key.key_data.app) {
apps[statement.key.key_data.app] = true;
} else {
sawBlank = true;
if (statement.key.key_data.app) {
apps[statement.key.key_data.app] = true;
} else {
sawBlank = true;
}
}
},
);
return []
.concat(sawInternal ? [sqlStatsState.data.internal_app_name_prefix] : [])
.concat(sawBlank ? ["(unset)"] : [])
.concat(Object.keys(apps));
return [].concat(sawBlank ? ["(unset)"] : []).concat(Object.keys(apps));
});

// selectDatabases returns the array of all databases with statement statistics present
Expand Down
3 changes: 1 addition & 2 deletions pkg/ui/workspaces/cluster-ui/src/transactionsPage/utils.ts
Original file line number Diff line number Diff line change
Expand Up @@ -39,14 +39,13 @@ export const getTrxAppFilterOptions = (
transactions: Transaction[],
prefix: string,
): string[] => {
const defaultAppFilters = [prefix];
const uniqueAppNames = new Set(
transactions
.filter(t => !t.stats_data.app.startsWith(prefix))
.map(t => (t.stats_data.app ? t.stats_data.app : "(unset)")),
);

return defaultAppFilters.concat(Array.from(uniqueAppNames));
return Array.from(uniqueAppNames);
};

export const collectStatementsText = (statements: Statement[]): string =>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -161,28 +161,27 @@ export const selectApps = createSelector(
}

let sawBlank = false;
let sawInternal = false;
const apps: { [app: string]: boolean } = {};
state.data.statements.forEach(
(statement: ICollectedStatementStatistics) => {
if (
const isNotInternalApp =
state.data.internal_app_name_prefix &&
statement.key.key_data.app.startsWith(
!statement.key.key_data.app.startsWith(
state.data.internal_app_name_prefix,
)
);
if (
state.data.internal_app_name_prefix == undefined ||
isNotInternalApp
) {
sawInternal = true;
} else if (statement.key.key_data.app) {
apps[statement.key.key_data.app] = true;
} else {
sawBlank = true;
if (statement.key.key_data.app) {
apps[statement.key.key_data.app] = true;
} else {
sawBlank = true;
}
}
},
);
return []
.concat(sawInternal ? [state.data.internal_app_name_prefix] : [])
.concat(sawBlank ? ["(unset)"] : [])
.concat(Object.keys(apps));
return [].concat(sawBlank ? ["(unset)"] : []).concat(Object.keys(apps));
},
);

Expand Down

0 comments on commit 1c4b0dd

Please sign in to comment.