Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
110901: ui: decouple SQL execution requests on databases page r=zachlite a=zachlite

Previously, SQL execution requests for database details and span stats were done in the same HTTP request. The consequence of this coupling was that no information could be displayed until span stats were collected. Since span stats are much slower to collect than database details, this resulted in the potential for a very bad user experience.

This commit separates span stats and database details into distinct HTTP requests, and updates the DatabasesPage component so that database details and span stats (and their associated network and query errors) can all be displayed independently.

Here is a screenshot of a simulated span stats error, and a simulated index recommendations error. Notice how the columns `Tables` and `Regions/Nodes` can still be displayed. 
![Screenshot 2023-09-15 at 9 49 19 AM](https://github.com/cockroachdb/cockroach/assets/5423191/0a6c353b-43b0-471a-8024-72ce10fe5a1f)


Epic: none
Release note (ui change): Added progressive loading functionality to the databases page.

112070: sql: allow refcursor comparison with NULL values r=DrewKimball a=DrewKimball

Most comparisons are disallowed for the REFCURSOR data type. For example, the following is not allowed in postgres:
```
SELECT 'foo'::REFCURSOR = 'bar'::REFCURSOR;
```
However, postgres does allow using `IS [NOT] DISTINCT FROM NULL` and `IS [NOT] NULL` with a refcursor-typed expression os the left operand. There are a few places where CRDB internally expects this comparison to be valid as well, so disallowing these comparisons has caused test failures.

This patch removes the refcursor-specific check for invalid comparisons during type checking, and relies on execution-time checks as is done for other comparisons. This means that `IsNull` and `IsNotNull` expressions can now be executed. In addition, the `IS DISTINCT FROM NULL` comparison is now supported only for the case of type `REFCURSOR` as the left operand, and `UNKNOWN` (NULL) as the right operand.

Fixes #112010
Fixes #112011

Release note: None

Co-authored-by: Zach Lite <[email protected]>
Co-authored-by: Drew Kimball <[email protected]>
  • Loading branch information
3 people committed Oct 9, 2023
3 parents cd1adae + 7f8d242 + 1c2d813 commit 559008f
Show file tree
Hide file tree
Showing 22 changed files with 634 additions and 199 deletions.
1 change: 1 addition & 0 deletions docs/generated/sql/operators.md
Original file line number Diff line number Diff line change
Expand Up @@ -447,6 +447,7 @@
<tr><td>oid <code>IS NOT DISTINCT FROM</code> <a href="int.html">int</a></td><td><a href="bool.html">bool</a></td></tr>
<tr><td>oid <code>IS NOT DISTINCT FROM</code> oid</td><td><a href="bool.html">bool</a></td></tr>
<tr><td>pg_lsn <code>IS NOT DISTINCT FROM</code> pg_lsn</td><td><a href="bool.html">bool</a></td></tr>
<tr><td>refcursor <code>IS NOT DISTINCT FROM</code> unknown</td><td><a href="bool.html">bool</a></td></tr>
<tr><td><a href="string.html">string</a> <code>IS NOT DISTINCT FROM</code> <a href="string.html">string</a></td><td><a href="bool.html">bool</a></td></tr>
<tr><td><a href="string.html">string[]</a> <code>IS NOT DISTINCT FROM</code> <a href="string.html">string[]</a></td><td><a href="bool.html">bool</a></td></tr>
<tr><td><a href="time.html">time</a> <code>IS NOT DISTINCT FROM</code> <a href="time.html">time</a></td><td><a href="bool.html">bool</a></td></tr>
Expand Down
106 changes: 97 additions & 9 deletions pkg/sql/logictest/testdata/logic_test/refcursor
Original file line number Diff line number Diff line change
Expand Up @@ -440,29 +440,117 @@ SELECT 'foo'::REFCURSOR >= 'foo'::TEXT;
statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR >= NULL;

# TODO(drewk): Postgres allows this case.
statement error pgcode 42883 pq: unsupported comparison operator
statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS DISTINCT FROM 'foo'::REFCURSOR;

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS DISTINCT FROM 'foo'::TEXT;

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM 'foo'::REFCURSOR;

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM 'foo'::TEXT;

# Regression test for #112010 - REFCURSOR should allow IS NULL and
# IS NOT NULL comparisons.
subtest is_null

query B
SELECT 'foo'::REFCURSOR IS NULL;
----
false

statement error pgcode 42883 pq: unsupported comparison operator
query B
SELECT 'foo'::REFCURSOR IS NOT NULL;
----
true

query B
SELECT 'foo'::REFCURSOR IS DISTINCT FROM NULL;
----
true

query B
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM NULL;
----
false

# REFCURSOR columns are allowed.
query B rowsort
SELECT l IS NULL FROM implicit_types;
----
true
true
true
true
true

query B rowsort
SELECT l IS NOT NULL FROM implicit_types;
----
false
false
false
false
false

query B rowsort
SELECT l IS DISTINCT FROM NULL FROM implicit_types;
----
false
false
false
false
false

query B rowsort
SELECT l IS NOT DISTINCT FROM NULL FROM implicit_types;
----
true
true
true
true
true

# Comparison with column is not allowed.
statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS DISTINCT FROM 'foo'::REFCURSOR;
SELECT 'foo'::REFCURSOR IS DISTINCT FROM l FROM implicit_types;

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS DISTINCT FROM 'foo'::TEXT;
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM l FROM implicit_types;

# Comparison with typed NULL is not allowed.
statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS DISTINCT FROM NULL;
SELECT 'foo'::REFCURSOR IS DISTINCT FROM (NULL::REFCURSOR);

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM 'foo'::REFCURSOR;
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM (NULL::REFCURSOR);

# Comparison with CASE expression is not allowed.
statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM 'foo'::TEXT;
SELECT 'foo'::REFCURSOR IS DISTINCT FROM (CASE WHEN true THEN NULL ELSE 1 END);

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM NULL;
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM (CASE WHEN true THEN NULL ELSE 1 END);

# The CASE operator is folded into an untyped NULL, so it's allowed.
# TODO(drewk): Postgres doesn't allow this case.
query B
SELECT 'foo'::REFCURSOR IS DISTINCT FROM (CASE WHEN true THEN NULL ELSE NULL END);
----
true

query B
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM (CASE WHEN true THEN NULL ELSE NULL END);
----
false

# The CASE operator is folded into a typed NULL, so it's not allowed.
statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS DISTINCT FROM (CASE WHEN true THEN NULL ELSE NULL::REFCURSOR END);

statement error pgcode 22023 pq: unsupported comparison operator
SELECT 'foo'::REFCURSOR IS NOT DISTINCT FROM (CASE WHEN true THEN NULL ELSE NULL::REFCURSOR END);

subtest end
4 changes: 4 additions & 0 deletions pkg/sql/sem/tree/eval.go
Original file line number Diff line number Diff line change
Expand Up @@ -1775,6 +1775,10 @@ var CmpOps = cmpOpFixups(map[treecmp.ComparisonOperatorSymbol]*CmpOpOverloads{
makeIsFn(types.Void, types.Unknown, volatility.Leakproof),
makeIsFn(types.Unknown, types.Void, volatility.Leakproof),

// REFCURSOR cannot be compared with itself, but as a special case, it can
// be compared with NULL using IS NOT DISTINCT FROM.
makeIsFn(types.RefCursor, types.Unknown, volatility.Leakproof),

// Tuple comparison.
{
LeftType: types.AnyTuple,
Expand Down
20 changes: 0 additions & 20 deletions pkg/sql/sem/tree/type_check.go
Original file line number Diff line number Diff line change
Expand Up @@ -941,14 +941,6 @@ func (expr *ComparisonExpr) TypeCheck(
return nil, err
}

// REFCURSOR does not support comparisons.
leftType, rightType := leftTyped.ResolvedType(), rightTyped.ResolvedType()
if leftType.Family() == types.RefCursorFamily || rightType.Family() == types.RefCursorFamily {
return nil, pgerror.Newf(pgcode.UndefinedFunction,
"unsupported comparison operator: <%s> %s <%s>", leftType, cmpOp, rightType,
)
}

if alwaysNull {
return DNull, nil
}
Expand Down Expand Up @@ -1471,12 +1463,6 @@ func (expr *IsNullExpr) TypeCheck(
if err != nil {
return nil, err
}
// REFCURSOR does not support comparisons.
if exprTyped.ResolvedType().Family() == types.RefCursorFamily {
return nil, pgerror.New(pgcode.UndefinedFunction,
"unsupported comparison operator: refcursor IS unknown",
)
}
expr.Expr = exprTyped
expr.typ = types.Bool
return expr, nil
Expand All @@ -1490,12 +1476,6 @@ func (expr *IsNotNullExpr) TypeCheck(
if err != nil {
return nil, err
}
// REFCURSOR does not support comparisons.
if exprTyped.ResolvedType().Family() == types.RefCursorFamily {
return nil, pgerror.New(pgcode.UndefinedFunction,
"unsupported comparison operator: refcursor IS NOT unknown",
)
}
expr.Expr = exprTyped
expr.typ = types.Bool
return expr, nil
Expand Down
131 changes: 84 additions & 47 deletions pkg/ui/workspaces/cluster-ui/src/api/databaseDetailsApi.ts
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ import {
SqlExecutionErrorMessage,
SqlExecutionRequest,
sqlResultsAreEmpty,
SqlExecutionResponse,
SqlStatement,
SqlTxnResult,
txnResultIsEmpty,
Expand All @@ -40,6 +41,10 @@ export type DatabaseDetailsReqParams = {
csIndexUnusedDuration: string;
};

export type DatabaseDetailsSpanStatsReqParams = {
database: string;
};

export type DatabaseDetailsResponse = {
idResp: SqlApiQueryResponse<DatabaseIdRow>;
grantsResp: SqlApiQueryResponse<DatabaseGrantsResponse>;
Expand All @@ -49,6 +54,11 @@ export type DatabaseDetailsResponse = {
error?: SqlExecutionErrorMessage;
};

export type DatabaseDetailsSpanStatsResponse = {
spanStats: SqlApiQueryResponse<DatabaseSpanStatsRow>;
error?: SqlExecutionErrorMessage;
};

function newDatabaseDetailsResponse(): DatabaseDetailsResponse {
return {
idResp: { database_id: "" },
Expand All @@ -62,12 +72,6 @@ function newDatabaseDetailsResponse(): DatabaseDetailsResponse {
zone_config_level: ZoneConfigurationLevel.CLUSTER,
},
stats: {
spanStats: {
approximate_disk_bytes: 0,
live_bytes: 0,
total_bytes: 0,
range_count: 0,
},
replicaData: {
replicas: [],
regions: [],
Expand All @@ -77,6 +81,18 @@ function newDatabaseDetailsResponse(): DatabaseDetailsResponse {
};
}

function newDatabaseDetailsSpanStatsResponse(): DatabaseDetailsSpanStatsResponse {
return {
spanStats: {
approximate_disk_bytes: 0,
live_bytes: 0,
total_bytes: 0,
range_count: 0,
},
error: undefined,
};
}

// Database ID
type DatabaseIdRow = {
database_id: string;
Expand Down Expand Up @@ -240,7 +256,7 @@ type DatabaseZoneConfigRow = {
const getDatabaseZoneConfig: DatabaseDetailsQuery<DatabaseZoneConfigRow> = {
createStmt: dbName => {
return {
sql: `SELECT
sql: `SELECT
encode(
crdb_internal.get_zone_config(
(SELECT crdb_internal.get_database_id($1))
Expand Down Expand Up @@ -293,7 +309,6 @@ const getDatabaseZoneConfig: DatabaseDetailsQuery<DatabaseZoneConfigRow> = {

// Database Stats
type DatabaseDetailsStats = {
spanStats: SqlApiQueryResponse<DatabaseSpanStatsRow>;
replicaData: SqlApiQueryResponse<DatabaseReplicasRegionsRow>;
indexStats: SqlApiQueryResponse<DatabaseIndexUsageStatsResponse>;
};
Expand All @@ -305,44 +320,38 @@ export type DatabaseSpanStatsRow = {
range_count: number;
};

const getDatabaseSpanStats: DatabaseDetailsQuery<DatabaseSpanStatsRow> = {
createStmt: dbName => {
return {
sql: `SELECT
sum(range_count) as range_count,
sum(approximate_disk_bytes) as approximate_disk_bytes,
sum(live_bytes) as live_bytes,
sum(total_bytes) as total_bytes
FROM crdb_internal.tenant_span_stats((SELECT crdb_internal.get_database_id($1)))`,
arguments: [dbName],
};
},
addToDatabaseDetail: (
txn_result: SqlTxnResult<DatabaseSpanStatsRow>,
resp: DatabaseDetailsResponse,
) => {
if (txn_result && txn_result.error) {
resp.stats.spanStats.error = txn_result.error;
}
if (txnResultIsEmpty(txn_result)) {
return;
}
if (txn_result.rows.length === 1) {
const row = txn_result.rows[0];
resp.stats.spanStats.approximate_disk_bytes = row.approximate_disk_bytes;
resp.stats.spanStats.range_count = row.range_count;
resp.stats.spanStats.live_bytes = row.live_bytes;
resp.stats.spanStats.total_bytes = row.total_bytes;
} else {
resp.stats.spanStats.error = new Error(
`DatabaseDetails - Span Stats, expected 1 row, got ${txn_result.rows.length}`,
);
}
},
handleMaxSizeError: (_dbName, _response, _dbDetail) => {
return Promise.resolve(false);
},
};
function formatSpanStatsExecutionResult(
res: SqlExecutionResponse<DatabaseSpanStatsRow>,
): DatabaseDetailsSpanStatsResponse {
const out = newDatabaseDetailsSpanStatsResponse();

if (res.execution.txn_results.length === 0) {
return out;
}

const txn_result = res.execution.txn_results[0];

if (txn_result && txn_result.error) {
// Copy the SQLExecutionError and the SqlTransactionResult error.
out.error = res.error;
out.spanStats.error = txn_result.error;
}
if (txnResultIsEmpty(txn_result)) {
return out;
}
if (txn_result.rows.length === 1) {
const row = txn_result.rows[0];
out.spanStats.approximate_disk_bytes = row.approximate_disk_bytes;
out.spanStats.range_count = row.range_count;
out.spanStats.live_bytes = row.live_bytes;
out.spanStats.total_bytes = row.total_bytes;
} else {
out.spanStats.error = new Error(
`DatabaseDetails - Span Stats, expected 1 row, got ${txn_result.rows.length}`,
);
}
return out;
}

type DatabaseReplicasRegionsRow = {
replicas: number[];
Expand Down Expand Up @@ -463,7 +472,6 @@ const databaseDetailQueries: DatabaseDetailsQuery<DatabaseDetailsRow>[] = [
getDatabaseReplicasAndRegions,
getDatabaseIndexUsageStats,
getDatabaseZoneConfig,
getDatabaseSpanStats,
];

export function createDatabaseDetailsReq(
Expand All @@ -480,6 +488,35 @@ export function createDatabaseDetailsReq(
};
}

export function createDatabaseDetailsSpanStatsReq(
params: DatabaseDetailsSpanStatsReqParams,
): SqlExecutionRequest {
const statement = {
sql: `SELECT
sum(range_count) as range_count,
sum(approximate_disk_bytes) as approximate_disk_bytes,
sum(live_bytes) as live_bytes,
sum(total_bytes) as total_bytes
FROM crdb_internal.tenant_span_stats((SELECT crdb_internal.get_database_id($1)))`,
arguments: [params.database],
};
return createSqlExecutionRequest(params.database, [statement]);
}

export async function getDatabaseDetailsSpanStats(
params: DatabaseDetailsSpanStatsReqParams,
) {
const req: SqlExecutionRequest = createDatabaseDetailsSpanStatsReq(params);
const sqlResp = await executeInternalSql<DatabaseSpanStatsRow>(req);
const res = formatSpanStatsExecutionResult(sqlResp);
return formatApiResult<DatabaseDetailsSpanStatsResponse>(
res,
res.error,
"retrieving database span stats",
false,
);
}

export async function getDatabaseDetails(
params: DatabaseDetailsReqParams,
timeout?: moment.Duration,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,7 +14,8 @@ import { Dispatch } from "redux";
import { databaseNameCCAttr } from "src/util/constants";
import { getMatchParamByName } from "src/util/query";
import { AppState } from "../store";
import { actions as databaseDetailsActions } from "../store/databaseDetails";
import { databaseDetailsReducer } from "../store/databaseDetails";
const databaseDetailsActions = databaseDetailsReducer.actions;
import {
actions as localStorageActions,
LocalStorageKeys,
Expand Down
Loading

0 comments on commit 559008f

Please sign in to comment.