Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
96386: importer: add deprecation notice for mysqldump/pgdump r=rafiss a=rafiss

fixes: #93660

Release note (sql change): The PGDUMP and MYSQLDUMP formats for IMPORT were marked as deprecated. They are still present, but will be removed in a future release. The page at https://www.cockroachlabs.com/docs/v22.2/migration-overview shows alternatives.

96675: roachtest: fix sqlalchemy version pinning r=e-mbrown a=rafiss

fixes #96090
backports needed for #96081 and #96059

The test setup was wrong, and was always using the latest sqlalchemy. This fixes the pinning, and also updates to a newer version.

Release note: None

96721: sql: fix r.end_key after comparison in show_range_for_row r=knz a=ecwall

Fixes #96714

Flip the `<` to `>` in `show_range_for_row` to be consistent with `show_ranges.go`:
```
    WHEN r.end_key < crdb_internal.table_span(%[1]d)[2] THEN '<after:'||crdb_internal.pretty_key(r.end_key,-1)||'>'
```
becomes
```
    WHEN r.end_key > crdb_internal.table_span(%[1]d)[2] THEN '<after:'||crdb_internal.pretty_key(r.end_key,-1)||'>'
```

Release note: None

96726: sql: use unimplemented error rather than syntax error for PL/pgSQL r=chengxiong-ruan,mgartner a=rafiss

fixes #96720

This change does not affect anything user-facing, but it allows us to parse CREATE FUNCTION statements that use `LANGUAGE plpgsql`. The plpgsql grammar is still not supported, so this will still show an error to the user. However, since parsing succeeds, the statement will be sent to telemetry logs, allowing us to analyze which parts of the plpgsql grammar that customers are trying to use.

Release note: None

Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Evan Wall <[email protected]>
  • Loading branch information
3 people committed Feb 7, 2023
5 parents f8a7f49 + 8c5959a + ef2f612 + 8e3c2fd + 6b34dfb commit 0a65a4e
Show file tree
Hide file tree
Showing 16 changed files with 174 additions and 65 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -262,7 +262,7 @@ ap-southeast-2 23
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE regional_by_row_table FOR ROW ('ap-southeast-2', 1)]
----
<before:/Table/57> <after:/Table/110/5>
<before:/Table/57>

query TIIII
SELECT crdb_region, pk, pk2, a, b FROM regional_by_row_table
Expand Down
31 changes: 4 additions & 27 deletions pkg/cmd/roachtest/tests/sqlalchemy.go
Original file line number Diff line number Diff line change
Expand Up @@ -30,14 +30,10 @@ import (
var sqlAlchemyResultRegex = regexp.MustCompile(`^(?P<test>test.*::.*::[^ \[\]]*(?:\[.*])?) (?P<result>\w+)\s+\[.+]$`)
var sqlAlchemyReleaseTagRegex = regexp.MustCompile(`^rel_(?P<major>\d+)_(?P<minor>\d+)_(?P<point>\d+)$`)

// TODO(arul): Investigate why we need this and can't install sql alchemy using
//
// pip.
var supportedSQLAlchemyTag = "rel_1_4_26"
var supportedSQLAlchemyTag = "1.4.46"

// This test runs the SQLAlchemy dialect test suite against a single Cockroach
// node.

func registerSQLAlchemy(r registry.Registry) {
r.Add(registry.TestSpec{
Name: "sqlalchemy",
Expand Down Expand Up @@ -98,10 +94,10 @@ func runSQLAlchemy(ctx context.Context, t test.Test, c cluster.Cluster) {
t.Fatal(err)
}

if err := repeatRunE(ctx, t, c, node, "install pytest", `
if err := repeatRunE(ctx, t, c, node, "install pytest", fmt.Sprintf(`
source venv/bin/activate &&
pip3 install --upgrade --force-reinstall setuptools pytest==6.0.1 pytest-xdist psycopg2 alembic
`); err != nil {
pip3 install --upgrade --force-reinstall setuptools pytest==6.0.1 pytest-xdist psycopg2 alembic sqlalchemy==%s`,
supportedSQLAlchemyTag)); err != nil {
t.Fatal(err)
}

Expand All @@ -124,25 +120,6 @@ func runSQLAlchemy(ctx context.Context, t test.Test, c cluster.Cluster) {
t.Fatal(err)
}

if err := repeatRunE(ctx, t, c, node, "remove old sqlalchemy", `
sudo rm -rf /mnt/data1/sqlalchemy
`); err != nil {
t.Fatal(err)
}

if err := repeatGitCloneE(ctx, t, c,
"https://github.com/sqlalchemy/sqlalchemy.git", "/mnt/data1/sqlalchemy",
supportedSQLAlchemyTag, node); err != nil {
t.Fatal(err)
}

t.Status("building sqlalchemy")
if err := repeatRunE(ctx, t, c, node, "building sqlalchemy", `
source venv/bin/activate && cd /mnt/data1/sqlalchemy && python3 setup.py build
`); err != nil {
t.Fatal(err)
}

// Phew, after having setup all that, let's actually run the test.

t.Status("setting up cockroach")
Expand Down
1 change: 1 addition & 0 deletions pkg/sql/catalog/funcdesc/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ go_library(
"//pkg/sql/sem/tree",
"//pkg/sql/sem/volatility",
"//pkg/sql/types",
"//pkg/util/errorutil/unimplemented",
"//pkg/util/hlc",
"//pkg/util/iterutil",
"//pkg/util/protoutil",
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/catalog/funcdesc/func_desc.go
Original file line number Diff line number Diff line change
Expand Up @@ -635,7 +635,7 @@ func (desc *immutable) getCreateExprLang() tree.FunctionLanguage {
case catpb.Function_SQL:
return tree.FunctionLangSQL
}
return 0
return tree.FunctionLangUnknown
}

func (desc *immutable) getCreateExprVolatility() tree.FunctionVolatility {
Expand Down
5 changes: 4 additions & 1 deletion pkg/sql/catalog/funcdesc/helpers.go
Original file line number Diff line number Diff line change
Expand Up @@ -15,6 +15,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgcode"
"github.com/cockroachdb/cockroach/pkg/sql/pgwire/pgerror"
"github.com/cockroachdb/cockroach/pkg/sql/sem/tree"
"github.com/cockroachdb/cockroach/pkg/util/errorutil/unimplemented"
)

// VolatilityToProto converts sql statement input volatility to protobuf
Expand Down Expand Up @@ -54,9 +55,11 @@ func FunctionLangToProto(v tree.FunctionLanguage) (catpb.Function_Language, erro
switch v {
case tree.FunctionLangSQL:
return catpb.Function_SQL, nil
case tree.FunctionLangPlPgSQL:
return -1, unimplemented.NewWithIssue(91569, "PL/pgSQL is not yet supported")
}

return -1, pgerror.Newf(pgcode.InvalidParameterValue, "Unknown function language %q", v)
return -1, pgerror.Newf(pgcode.UndefinedObject, "language %q does not exist", v)
}

// ParamClassToProto converts sql statement input argument class to protobuf
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/delegate/show_range_for_row.go
Original file line number Diff line number Diff line change
Expand Up @@ -69,7 +69,7 @@ SELECT
END AS start_key,
CASE
WHEN r.end_key = crdb_internal.table_span(%[1]d)[2] THEN '…/<TableMax>'
WHEN r.end_key < crdb_internal.table_span(%[1]d)[2] THEN '<after:'||crdb_internal.pretty_key(r.end_key,-1)||'>'
WHEN r.end_key > crdb_internal.table_span(%[1]d)[2] THEN '<after:'||crdb_internal.pretty_key(r.end_key,-1)||'>'
ELSE '…'||crdb_internal.pretty_key(r.end_key, 2)
END AS end_key,
range_id,
Expand Down
2 changes: 2 additions & 0 deletions pkg/sql/importer/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -35,6 +35,7 @@ go_library(
"//pkg/cloud/cloudprivilege",
"//pkg/clusterversion",
"//pkg/col/coldata",
"//pkg/docs",
"//pkg/featureflag",
"//pkg/geo",
"//pkg/geo/geopb",
Expand Down Expand Up @@ -244,6 +245,7 @@ go_test(
"@com_github_fraugster_parquet_go//parquet",
"@com_github_go_sql_driver_mysql//:mysql",
"@com_github_gogo_protobuf//proto",
"@com_github_jackc_pgconn//:pgconn",
"@com_github_jackc_pgx_v4//:pgx",
"@com_github_kr_pretty//:pretty",
"@com_github_lib_pq//:pq",
Expand Down
9 changes: 9 additions & 0 deletions pkg/sql/importer/import_planning.go
Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,7 @@ import (

"github.com/cockroachdb/cockroach/pkg/cloud"
"github.com/cockroachdb/cockroach/pkg/cloud/cloudprivilege"
"github.com/cockroachdb/cockroach/pkg/docs"
"github.com/cockroachdb/cockroach/pkg/featureflag"
"github.com/cockroachdb/cockroach/pkg/jobs"
"github.com/cockroachdb/cockroach/pkg/jobs/jobspb"
Expand Down Expand Up @@ -363,6 +364,14 @@ func importPlanHook(
p.BufferClientNotice(ctx, pgnotice.Newf("IMPORT TABLE has been deprecated in 21.2, and will be removed in a future version."+
" Instead, use CREATE TABLE with the desired schema, and IMPORT INTO the newly created table."))
}
switch f := strings.ToUpper(importStmt.FileFormat); f {
case "PGDUMP", "MYSQLDUMP":
p.BufferClientNotice(ctx, pgnotice.Newf(
"IMPORT %s has been deprecated in 23.1, and will be removed in a future version. See %s for alternatives.",
redact.SafeString(f),
redact.SafeString(docs.URL("migration-overview")),
))
}

addToFileFormatTelemetry(importStmt.FileFormat, "attempted")

Expand Down
33 changes: 31 additions & 2 deletions pkg/sql/importer/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -78,6 +78,7 @@ import (
"github.com/cockroachdb/cockroach/pkg/util/syncutil"
"github.com/cockroachdb/cockroach/pkg/util/timeutil"
"github.com/cockroachdb/errors"
"github.com/jackc/pgconn"
"github.com/jackc/pgx/v4"
"github.com/linkedin/goavro/v2"
"github.com/stretchr/testify/assert"
Expand Down Expand Up @@ -1684,6 +1685,23 @@ func TestImportRowLimit(t *testing.T) {
conn := tc.ServerConn(0)
sqlDB := sqlutils.MakeSQLRunner(conn)

// Also create a pgx connection so we can check notices.
pgURL, cleanup := sqlutils.PGUrl(
t,
tc.Server(0).ServingSQLAddr(),
"TestImportRowLimit",
url.User(username.RootUser),
)
defer cleanup()
config, err := pgx.ParseConfig(pgURL.String())
require.NoError(t, err)
var noticeMsg string
config.OnNotice = func(_ *pgconn.PgConn, notice *pgconn.Notice) {
noticeMsg = notice.Message
}
pgxConn, err := pgx.ConnectConfig(ctx, config)
require.NoError(t, err)

avroField := []map[string]interface{}{
{
"name": "a",
Expand Down Expand Up @@ -1864,14 +1882,25 @@ func TestImportRowLimit(t *testing.T) {

// Import table from dump format.
importDumpQuery := fmt.Sprintf(`IMPORT TABLE t FROM %s ($1) %s`, test.typ, test.with)
sqlDB.Exec(t, importDumpQuery, srv.URL)
_, err := pgxConn.Exec(ctx, importDumpQuery, srv.URL)
require.NoError(t, err)
require.Regexp(t, fmt.Sprintf(
"IMPORT %s has been deprecated in 23.1.*See https://www.cockroachlabs.com/docs/.*/migration-overview for alternatives.",
test.typ,
), noticeMsg)

sqlDB.CheckQueryResults(t, test.verifyQuery, test.expected)

sqlDB.Exec(t, `DROP TABLE t`)

// Import dump format directly.
importDumpQuery = fmt.Sprintf(`IMPORT %s ($1) %s`, test.typ, test.with)
sqlDB.Exec(t, importDumpQuery, srv.URL)
_, err = pgxConn.Exec(ctx, importDumpQuery, srv.URL)
require.NoError(t, err)
require.Regexp(t, fmt.Sprintf(
"IMPORT %s has been deprecated in 23.1.*See https://www.cockroachlabs.com/docs/.*/migration-overview for alternatives.",
test.typ,
), noticeMsg)
sqlDB.CheckQueryResults(t, test.verifyQuery, test.expected)

sqlDB.Exec(t, `DROP TABLE t`)
Expand Down
37 changes: 25 additions & 12 deletions pkg/sql/logictest/testdata/logic_test/ranges
Original file line number Diff line number Diff line change
Expand Up @@ -364,10 +364,23 @@ CREATE TABLE simple_range_for_row(x INT PRIMARY KEY)
statement ok
ALTER TABLE simple_range_for_row SPLIT AT VALUES (1), (2)

# Before split
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE simple_range_for_row FOR ROW (0)]
----
<before:/Table/113/2/0> …/1

# Between splits
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE simple_range_for_row FOR ROW (1)]
----
…/1 <after:/Table/120/1/2>
…/1 …/2

# After split
query TT retry
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE simple_range_for_row FOR ROW (2)]
----
…/2 <after:/Max>

statement ok
CREATE TABLE range_for_row(x INT, y INT, z INT, w INT, PRIMARY KEY (x, y), INDEX i (z, w))
Expand All @@ -381,32 +394,32 @@ ALTER INDEX range_for_row@i SPLIT AT VALUES (3, 4), (3, 5)
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row FOR ROW (1, 2)]
----
…/1/2 <after:/Table/121/1/1/3>
…/1/2 …/1/3

query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row FOR ROW (1, 3)]
----
…/1/3 <after:/Table/121/2/3/4>
…/1/3 …/3/4

query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row FOR ROW (1, 1)]
----
<before:/Table/120/1/2> <after:/Table/121/1/1/2>
<before:/Table/120/1/2> …/1/2

query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM INDEX range_for_row@i FOR ROW (1, 2, 1, 2)]
----
…/1/3 <after:/Table/121/2/3/4>
…/1/3 …/3/4

query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM INDEX range_for_row@i FOR ROW (3, 4, 1, 2)]
----
…/3/4 <after:/Table/121/2/3/5>
…/3/4 …/3/5

query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM INDEX range_for_row@i FOR ROW (3, 5, 1, 2)]
----
…/3/5
…/3/5 <after:/Max>

statement ok
CREATE TABLE range_for_row_string(x STRING PRIMARY KEY)
Expand All @@ -417,7 +430,7 @@ ALTER TABLE range_for_row_string SPLIT AT VALUES ('hello')
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row_string FOR ROW ('he')]
----
<before:/Table/121/2/3/5> <after:/Table/122/1/"hello">
<before:/Table/121/2/3/5> …/"hello"

statement ok
CREATE TABLE range_for_row_decimal(x DECIMAL PRIMARY KEY)
Expand All @@ -428,7 +441,7 @@ ALTER TABLE range_for_row_decimal SPLIT AT VALUES (1), (2)
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE range_for_row_decimal FOR ROW (1)]
----
…/1 <after:/Table/123/1/2>
…/1 …/2

statement ok
CREATE TABLE range_for_row_nulls(x INT PRIMARY KEY, y INT, INDEX i (y))
Expand All @@ -439,7 +452,7 @@ ALTER INDEX range_for_row_nulls@i SPLIT AT VALUES (NULL)
query TT
SELECT start_key, end_key from [SHOW RANGE FROM INDEX range_for_row_nulls@i FOR ROW (NULL, 1)]
----
…/NULL
…/NULL <after:/Max>

subtest end

Expand Down Expand Up @@ -529,12 +542,12 @@ ALTER TABLE t63646 SPLIT AT VALUES ('a'), ('b')
query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE t63646 FOR ROW ('a')]
----
…/"@" <after:/Table/130/1/"\x80">
…/"@" …/"\x80"

query TT
SELECT start_key, end_key FROM [SHOW RANGE FROM TABLE t63646 FOR ROW ('b')]
----
…/"\x80"
…/"\x80" <after:/Max>

# Test permissions for showing ranges with ZONECONFIG privilege

Expand Down
18 changes: 18 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/udf
Original file line number Diff line number Diff line change
Expand Up @@ -4,6 +4,24 @@ CREATE TABLE ab (
b INT
)

statement error pgcode 0A000 unimplemented: PL/pgSQL is not yet supported
CREATE FUNCTION populate() RETURNS integer AS $$
DECLARE
-- declarations
BEGIN
PERFORM my_function();
END;
$$ LANGUAGE plpgsql

statement error pgcode 42704 language \"made_up_language\" does not exist
CREATE FUNCTION populate() RETURNS integer AS $$
DECLARE
-- declarations
BEGIN
PERFORM my_function();
END;
$$ LANGUAGE made_up_language

statement error pq: unimplemented: user-defined functions with SETOF return types are not supported
CREATE FUNCTION f(a int) RETURNS SETOF INT LANGUAGE SQL AS 'SELECT 1'

Expand Down
1 change: 1 addition & 0 deletions pkg/sql/opt/optbuilder/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -52,6 +52,7 @@ go_library(
"//pkg/settings",
"//pkg/sql/catalog/colinfo",
"//pkg/sql/catalog/descpb",
"//pkg/sql/catalog/funcdesc",
"//pkg/sql/catalog/schemaexpr",
"//pkg/sql/catalog/seqexpr",
"//pkg/sql/catalog/tabledesc",
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/opt/optbuilder/create_function.go
Original file line number Diff line number Diff line change
Expand Up @@ -12,6 +12,7 @@ package optbuilder

import (
"github.com/cockroachdb/cockroach/pkg/sql/catalog/descpb"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/funcdesc"
"github.com/cockroachdb/cockroach/pkg/sql/catalog/typedesc"
"github.com/cockroachdb/cockroach/pkg/sql/opt"
"github.com/cockroachdb/cockroach/pkg/sql/opt/memo"
Expand Down Expand Up @@ -81,6 +82,10 @@ func (b *Builder) buildCreateFunction(cf *tree.CreateFunction, inScope *scope) (
funcBodyStr = string(opt)
case tree.FunctionLanguage:
languageFound = true
// Check the language here, before attempting to parse the function body.
if _, err := funcdesc.FunctionLangToProto(opt); err != nil {
panic(err)
}
}
}

Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/opt/testutils/testcat/function.go
Original file line number Diff line number Diff line change
Expand Up @@ -146,8 +146,8 @@ func collectFuncOptions(
}

case tree.FunctionLanguage:
if t != tree.FunctionLangSQL {
panic(fmt.Errorf("LANGUAGE must be SQL"))
if t != tree.FunctionLangSQL && t != tree.FunctionLangPlPgSQL {
panic(fmt.Errorf("LANGUAGE must be SQL or plpgsql"))
}

default:
Expand Down
Loading

0 comments on commit 0a65a4e

Please sign in to comment.