Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
74863: import: check readability earlier r=benbardin a=benbardin

Release note (sql change): Import now checks readability earlier for multiple files, to fail sooner if e.g. permissions are invalid.

74914: opt,tree: fix bugs with Next(), Prev(), and histogram calculation for DTimeTZ r=rytaft a=rytaft

**sql/sem/tree: fix Next() and Prev() for DTimeTZ**

Prior to this commit, the `DTimeTZ` functions `Next()` and `Prev()`
could skip over valid values according to the ordering of `DTimeTZ`
values in an index (which matches the ordering defined by the
`TimeTZ` functions `After()` and `Before()`).

This commit fixes these functions so that `Next()` now returns the smallest
valid `DTimeTZ` that is greater than the receiver, and `Prev()` returns
the largest valid `DTimeTZ` that is less than the receiver. This is
an important invariant that the optimizer relies on when building index
constraints.

Fixes #74912

Release note (bug fix): Fixed a bug that could occur when a `TIMETZ`
column was indexed, and a query predicate constrained that column using
a `<` or `>` operator with a `timetz` constant. If the column contained values
with time zones that did not match the time zone of the `timetz` constant,
it was possible that not all matching values could be returned by the
query. Specifically, the results may not have included values within one
microsecond of the predicate's absolute time. This bug was introduced
when the timetz datatype was first added in 20.1. It exists on all
versions of 20.1, 20.2, 21.1, and 21.2 prior to this patch.

**opt: fix bug in histogram calculation for TimeTZ**

This commit fixes a bug in the histogram estimation code for `TimeTZ`
that made the faulty assumption that `TimeTZ` values are ordered by
`TimeOfDay`. This is incorrect since it does not take the `OffsetSecs`
into account. As a result, it was possible to estimate that the size
of a histogram bucket was negative, which caused problems in the
statistics estimation code. This commit fixes the problem by taking
into account both `TimeOfDay` and `OffsetSecs` when estimating the size of
a bucket in a `TimeTZ` histogram.

Fixes #74667

Release note (bug fix): Fixed an internal error, "estimated row count must
be non-zero", that could occur during planning for queries over a table
with a `TimeTZ` column. This error was due to a faulty assumption in the
statistics estimation code about ordering of `TimeTZ` values, which has now
been fixed. The error could occur when `TimeTZ` values used in the query had
a different time zone offset than the `TimeTZ` values stored in the table.

75112: sql: fix casts between REG* types r=mgartner a=mgartner

The newly introduced `castMap` does not contain entries for casts
between all combinations of REG* types, which is consistent with
Postgres, but inconsistent with behavior in versions up to 21.2 where
these casts are allowed.

The `castMap` changes result in more than just backward incompatibility.
We allow branches of CASE statements to be equivalent types (i.e., types
in the same family), like `REGCLASS` and `REGTYPE`, and we automatically
add casts to a query plan to support this. However, because these casts
don't exist in the `castMap`, internal errors are raised when we try to
fetch the volatility of the cast while building logical properties.

According to Postgres's type conversion rules for CASE, we should only
allow branches to be different types if they can be implicitly cast to
the first non-NULL branch. Implicit casts between REG* types are not
allowed, so CASE expressions with branches of different REG* types
should result in a user error like `CASE/WHEN could not convert type
regclass to regtype`. However, this is a much larger project and the
change will not be fully backward compatible. This work is tracked by
issue #75103.

For now, this commit adds casts between REG* types to the `castMap` to
maintain backward compatibility and prevent an internal error.

There is no release note because this bug does not exist in any
releases.

Fixes #74784

Release note: None

75119: sql: deflake TestPerfLogging r=rytaft a=rytaft

This commit deflakes `TestPerfLogging` by ensuring that test cases
that should not produce log entries do not match with unrelated log
entries and thus cause the test to fail. This is ensured by making
the regex more precise for the specific test case.

Fixes #74811

Release note: None

75146: backupccl: "skip" TestChangefeedRestartDuringBackfill.. r=irfansharif a=irfansharif

under span configs. This test flakes pretty reliably after span configs
were enabled (#73876). Investigating this further is being tracked in
\#75080; lets have this test use the old subsystem for now (only down in
KV; we've narrowed down the failure to having something to do with
concurrent range splits, within the tenant keyspace, while a changefeed
is declared).

Release note: None

Co-authored-by: Ben Bardin <bardin@cockroachlabs.com>
Co-authored-by: Rebecca Taft <becca@cockroachlabs.com>
Co-authored-by: Marcus Gartner <marcus@cockroachlabs.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
  • Loading branch information
5 people committed Jan 19, 2022
6 parents bed5b79 + 16aecb2 + ad7bf04 + d09d141 + 311c2e0 + c07bc46 commit 419167d
Show file tree
Hide file tree
Showing 15 changed files with 566 additions and 79 deletions.
18 changes: 13 additions & 5 deletions pkg/ccl/changefeedccl/changefeed_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -394,12 +394,12 @@ func TestChangefeedFullTableName(t *testing.T) {
assertPayloads(t, foo, []string{`d.public.foo: [1]->{"after": {"a": 1, "b": "a"}}`})
})
}
//TODO(zinger): Plumb this option through to all encoders so it works in sinkless mode
//t.Run(`sinkless`, sinklessTest(testFn))
// TODO(zinger): Plumb this option through to all encoders so it works in sinkless mode
// t.Run(`sinkless`, sinklessTest(testFn))
t.Run(`enterprise`, enterpriseTest(testFn))
t.Run(`kafka`, kafkaTest(testFn))
t.Run(`webhook`, webhookTest(testFn))
//t.Run(`pubsub`, pubsubTest(testFn))
// t.Run(`pubsub`, pubsubTest(testFn))
}

func TestChangefeedMultiTable(t *testing.T) {
Expand Down Expand Up @@ -2417,7 +2417,7 @@ func TestChangefeedTruncateOrDrop(t *testing.T) {
t.Run(`kafka`, kafkaTest(testFn))
t.Run(`webhook`, webhookTest(testFn))
t.Run(`pubsub`, pubsubTest(testFn))
//will sometimes fail, non deterministic
// will sometimes fail, non deterministic
}

func TestChangefeedMonitoring(t *testing.T) {
Expand Down Expand Up @@ -4102,7 +4102,15 @@ func TestChangefeedRestartDuringBackfill(t *testing.T) {
})
}

t.Run(`kafka`, kafkaTest(testFn))
useSysCfgInKV := withKnobsFn(func(knobs *base.TestingKnobs) {
// TODO(irfansharif): This test is "skipped" under span configs;
// #75080.
if knobs.Store == nil {
knobs.Store = &kvserver.StoreTestingKnobs{}
}
knobs.Store.(*kvserver.StoreTestingKnobs).UseSystemConfigSpanForQueues = true
})
t.Run(`kafka`, kafkaTest(testFn, useSysCfgInKV))
}

func TestChangefeedHandlesDrainingNodes(t *testing.T) {
Expand Down
9 changes: 3 additions & 6 deletions pkg/ccl/importccl/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -7043,15 +7043,12 @@ func TestImportRowErrorLargeRows(t *testing.T) {
if r.Method != "GET" {
return
}
_, err := w.Write([]byte("firstrowvalue\nsecondrow,is,notok,"))
require.NoError(t, err)
_, _ = w.Write([]byte("firstrowvalue\nsecondrow,is,notok,"))
// Write 8MB field as the last field of the second
// row.
bigData := randutil.RandBytes(rng, 8<<20)
_, err = w.Write(bigData)
require.NoError(t, err)
_, err = w.Write([]byte("\n"))
require.NoError(t, err)
_, _ = w.Write(bigData)
_, _ = w.Write([]byte("\n"))
}))
defer srv.Close()
tc := testcluster.StartTestCluster(t, 1, base.TestClusterArgs{})
Expand Down
65 changes: 50 additions & 15 deletions pkg/ccl/importccl/read_import_base.go
Original file line number Diff line number Diff line change
Expand Up @@ -153,24 +153,59 @@ func readInputFiles(

fileSizes := make(map[int32]int64, len(dataFiles))

// Attempt to fetch total number of bytes for all files.
// "Pre-import" work.
// Validate readability early, and attempt to fetch total number of bytes for
// all files to track progress.
for id, dataFile := range dataFiles {
conf, err := cloud.ExternalStorageConfFromURI(dataFile, user)
if err != nil {
return err
}
es, err := makeExternalStorage(ctx, conf)
if err != nil {
if err := func() error {
// Run within an anonymous function to release each connection after each
// iteration, rather than all at once after the `for` loop.
conf, err := cloud.ExternalStorageConfFromURI(dataFile, user)
if err != nil {
return err
}
es, err := makeExternalStorage(ctx, conf)
if err != nil {
return err
}
defer es.Close()

sz, err := es.Size(ctx, "")

if sz <= 0 {
// Don't log dataFile here because it could leak auth information.
log.Infof(ctx, "could not fetch file size; falling back to per-file progress: %v", err)
} else {
fileSizes[id] = sz
}

if len(dataFiles) > 1 {
// If there's more than one file, try to read a byte from each to verify
// readability (e.g. permissions).
// If there's only one file, skip that check because it provides no
// advantage.
raw, err := es.ReadFile(ctx, "")
if err != nil {
return err
}
defer raw.Close()

p := make([]byte, 1)
if _, err := raw.Read(p); err != nil && err != io.EOF {
// Check that we can read the file. We don't care about content yet,
// so we read a single byte and we don't process it in any way.
// If the file is empty -- and we can tell that -- that also counts
// as readable, so don't error.
return err
}

}

return nil
}(); err != nil {
return err
}
sz, err := es.Size(ctx, "")
es.Close()
if sz <= 0 {
// Don't log dataFile here because it could leak auth information.
log.Infof(ctx, "could not fetch file size; falling back to per-file progress: %v", err)
break
}
fileSizes[id] = sz

}

for dataFileIndex, dataFile := range dataFiles {
Expand Down
28 changes: 14 additions & 14 deletions pkg/sql/event_log_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -409,7 +409,7 @@ func TestPerfLogging(t *testing.T) {
{
query: `INSERT INTO t(i) VALUES (6)`,
errRe: ``,
logRe: `"EventType":"txn_rows_written_limit"`,
logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
Expand Down Expand Up @@ -483,14 +483,14 @@ func TestPerfLogging(t *testing.T) {
cleanup: `RESET transaction_rows_written_err`,
query: `INSERT INTO t(i) VALUES (15), (16)`,
errRe: `pq: txn has written 2 rows, which is above the limit: TxnID .* SessionID .*`,
logRe: `"EventType":"txn_rows_written_limit"`,
logRe: `"EventType":"txn_rows_written_limit","Statement":"INSERT INTO.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
query: `SELECT * FROM t WHERE i = 6`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT \* FROM .*‹t› WHERE ‹i› = ‹6›","Tag":"SELECT","User":"root","TxnID":.*,"SessionID":.*`,
logExpected: false,
channel: channel.SQL_PERF,
},
Expand Down Expand Up @@ -540,7 +540,7 @@ func TestPerfLogging(t *testing.T) {
cleanup: `RESET transaction_rows_read_err`,
query: `SELECT * FROM t WHERE i = 6 OR i = 7`,
errRe: `pq: txn has read 2 rows, which is above the limit: TxnID .* SessionID .*`,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"SELECT \* FROM .*‹t› WHERE ‹i› = ‹6› OR ‹i› = ‹7›","Tag":"SELECT","User":"root","TxnID":.*,"SessionID":.*`,
logExpected: false,
channel: channel.SQL_PERF,
},
Expand All @@ -559,77 +559,77 @@ func TestPerfLogging(t *testing.T) {
cleanup: `DROP TABLE t_copy`,
query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
errRe: ``,
logRe: `"EventType":"txn_rows_written_limit"`,
logRe: `"EventType":"txn_rows_written_limit","Statement":"CREATE.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
cleanup: `DROP TABLE t_copy`,
query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"CREATE.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
cleanup: `DROP TABLE t_copy`,
query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
errRe: ``,
logRe: `"EventType":"txn_rows_written_limit"`,
logRe: `"EventType":"txn_rows_written_limit","Statement":"CREATE.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_INTERNAL_PERF,
},
{
cleanup: `DROP TABLE t_copy`,
query: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"CREATE.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_INTERNAL_PERF,
},
{
setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
query: `DROP TABLE t_copy`,
errRe: ``,
logRe: `"EventType":"txn_rows_written_limit"`,
logRe: `"EventType":"txn_rows_written_limit","Statement":"DROP.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
query: `DROP TABLE t_copy`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"DROP.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
query: `DROP TABLE t_copy`,
errRe: ``,
logRe: `"EventType":"txn_rows_written_limit"`,
logRe: `"EventType":"txn_rows_written_limit","Statement":"DROP.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_INTERNAL_PERF,
},
{
setup: `CREATE TABLE t_copy (i PRIMARY KEY) AS SELECT i FROM t`,
query: `DROP TABLE t_copy`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"DROP.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_INTERNAL_PERF,
},
{
query: `ANALYZE t`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"ANALYZE.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_PERF,
},
{
query: `ANALYZE t`,
errRe: ``,
logRe: `"EventType":"txn_rows_read_limit"`,
logRe: `"EventType":"txn_rows_read_limit","Statement":"ANALYZE.*","TxnID":".*","SessionID":".*"`,
logExpected: false,
channel: channel.SQL_INTERNAL_PERF,
},
Expand Down
37 changes: 37 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/cast
Original file line number Diff line number Diff line change
Expand Up @@ -1342,3 +1342,40 @@ INSERT INTO t59489 (d4_2) VALUES (600)

statement error value with precision 4, scale 2 must round to an absolute value less than 10\^2
INSERT INTO t59489 (d4_2) VALUES (6e2)

# Regression for #74784. Case expressions with branches of different types in
# the OID family should not cause internal errors.
# TODO(#75103): These CASE expressions are allowed for backward compatibility,
# but ultimately should not allow expressions such as these because implicit
# casts between REG* are not allowed.
statement ok
select CASE WHEN false THEN 1::REGCLASS ELSE 2::REGNAMESPACE END;
select CASE WHEN false THEN 1::REGCLASS ELSE 2::REGPROC END;
select CASE WHEN false THEN 1::REGCLASS ELSE 2::REGPROCEDURE END;
select CASE WHEN false THEN 1::REGCLASS ELSE 2::REGROLE END;
select CASE WHEN false THEN 1::REGCLASS ELSE 2::REGTYPE END;
select CASE WHEN false THEN 1::REGNAMESPACE ELSE 2::REGCLASS END;
select CASE WHEN false THEN 1::REGNAMESPACE ELSE 2::REGPROC END;
select CASE WHEN false THEN 1::REGNAMESPACE ELSE 2::REGPROCEDURE END;
select CASE WHEN false THEN 1::REGNAMESPACE ELSE 2::REGROLE END;
select CASE WHEN false THEN 1::REGNAMESPACE ELSE 2::REGTYPE END;
select CASE WHEN false THEN 1::REGPROC ELSE 2::REGCLASS END;
select CASE WHEN false THEN 1::REGPROC ELSE 2::REGNAMESPACE END;
select CASE WHEN false THEN 1::REGPROC ELSE 2::REGPROCEDURE END;
select CASE WHEN false THEN 1::REGPROC ELSE 2::REGROLE END;
select CASE WHEN false THEN 1::REGPROC ELSE 2::REGTYPE END;
select CASE WHEN false THEN 1::REGPROCEDURE ELSE 2::REGCLASS END;
select CASE WHEN false THEN 1::REGPROCEDURE ELSE 2::REGNAMESPACE END;
select CASE WHEN false THEN 1::REGPROCEDURE ELSE 2::REGPROC END;
select CASE WHEN false THEN 1::REGPROCEDURE ELSE 2::REGROLE END;
select CASE WHEN false THEN 1::REGPROCEDURE ELSE 2::REGTYPE END;
select CASE WHEN false THEN 1::REGROLE ELSE 2::REGCLASS END;
select CASE WHEN false THEN 1::REGROLE ELSE 2::REGNAMESPACE END;
select CASE WHEN false THEN 1::REGROLE ELSE 2::REGPROC END;
select CASE WHEN false THEN 1::REGROLE ELSE 2::REGPROCEDURE END;
select CASE WHEN false THEN 1::REGROLE ELSE 2::REGTYPE END;
select CASE WHEN false THEN 1::REGTYPE ELSE 2::REGCLASS END;
select CASE WHEN false THEN 1::REGTYPE ELSE 2::REGNAMESPACE END;
select CASE WHEN false THEN 1::REGTYPE ELSE 2::REGPROC END;
select CASE WHEN false THEN 1::REGTYPE ELSE 2::REGPROCEDURE END;
select CASE WHEN false THEN 1::REGTYPE ELSE 2::REGROLE END;
35 changes: 35 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/timetz
Original file line number Diff line number Diff line change
Expand Up @@ -670,3 +670,38 @@ SELECT a FROM regression_44774 ORDER BY a

statement ok
DROP TABLE regression_44774

# Check that index scans of timetz columns do not miss values.
subtest regression_74912

# Create a table with two identical columns. One is indexed, and one is not.
statement ok
CREATE TABLE regression_74912 (t1 TIMETZ PRIMARY KEY, t2 TIMETZ);
INSERT INTO regression_74912 VALUES
('05:00:00.000001', '05:00:00.000001'),
('07:00:00.000001+02:00:00', '07:00:00.000001+02:00:00'),
('09:00:00.000001+04:00:00', '09:00:00.000001+04:00:00'),
('20:59:00.000001+15:59:00', '20:59:00.000001+15:59:00');

query I
SELECT count(*) FROM regression_74912@{NO_FULL_SCAN} WHERE t1 > '05:00:00';
----
4

query I
SELECT count(*) FROM regression_74912@{NO_FULL_SCAN} WHERE t1 < '05:00:00.000001';
----
3

query I
SELECT count(*) FROM regression_74912 WHERE t2 > '05:00:00';
----
4

query I
SELECT count(*) FROM regression_74912 WHERE t2 < '05:00:00.000001';
----
3

statement ok
DROP TABLE regression_74912
Loading

0 comments on commit 419167d

Please sign in to comment.