Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
85850: importer: support {} format for arrays in CSV r=ecwall a=rafiss

fixes #84631

Release note (sql change): Arrays can now be imported in a CSV file
using the {} format, similar to COPY FROM. Importing array expressions
(e.g. ARRAY[1, 2, 3]) is still supported as well.

85854: clusterversion,storage: remove 22.1 PebbleFormat version gates r=jbowens a=celiala

This commit removes the following 22.1 version gates:

- PebbleFormatBlockPropertyCollector
- PebbleFormatSplitUserKeysMarked

Cleanup was done following guidance from [21.2 cleanup](#74270 (comment)):

> For the most part, if the gates were just simple if !version.IsActive { return x } or something, I just removed the block, and even if it was a little more complicated, like args = [x]; if version { args = append(args, y) }; foo(args) I still tried to mostly inline it such that it looked natural (i.e. remove that append and make it args = [x, y]).

> However for just a couple more complicated cases that were referring to <21.2 versions that needed to be replaced when those were deleted, I added a placeholder clusterversion.TODOPre21_2 alias for 21.2. Replacing those calls with this alias shouldn't change their behavior -- it was already always true, since the code today should never run in a <21.2 cluster -- but means we can delete those older versions in the meantime and then the owners of these bits can decide how to update them.

Partially addresses #80663

Release note: none

85909: kvserver: instrument RaftTransport workers with pprof labels r=tbg,erikgrinaker a=pavelkalinnikov

The unused arguments in the method signature were used to identify goroutines
in traces. This no longer works after Go 1.17 started passing arguments via
registers.

This commit adds pprof labels when starting these goroutines, to have a cleaner
code, more readable traces, and to work around the new Go convention.

Release note: None

85977: grunning: add library for precise on-CPU time measurement r=irfansharif a=irfansharif

Package grunning is a library that's able to retrieve on-CPU running
time for individual goroutines. It relies on using a patched Go and
provides a primitive for fine-grained CPU attribution and control
through a single API:

    package grunning

    // Time returns the time spent by the current goroutine in the
    // running state.
    func Time() time.Duration

The motivating RFC is over at #82356. Informs #82625.
We build CRDB using use the patched Go runtime for all officially
supported platforms when built using Bazel (#84867). Engineers commonly
building CRDB also use happen to use two platforms we don't use a
patched Go for:
- FreeBSD (we don't have cross-compilers setup), and
- M1/M2 Macs (we don't have a code-signing pipeline, yet).
We use '(darwin && arm64) || freebsd || !bazel' as the build tag to
exclude such platforms. See #84867 for more details.

This package tests various properties we should expect over the running time
value. It does not make assertions given the CI environments we run these
under (CPU-starved, lot of OS thread pre-emption, dissimilar to healthy
CRDB deployments). This is also why they're skipped under stress. Still,
these tests are useful to understand the properties we expect running
time to have:

    === RUN   TestEquivalentGoroutines
        thread=03 expected≈10.00% got= 9.98% of on-cpu time
        thread=06 expected≈10.00% got=10.00% of on-cpu time
        thread=02 expected≈10.00% got=10.01% of on-cpu time
        thread=10 expected≈10.00% got=10.01% of on-cpu time
        thread=07 expected≈10.00% got= 9.99% of on-cpu time
        thread=04 expected≈10.00% got= 9.99% of on-cpu time
        thread=09 expected≈10.00% got=10.00% of on-cpu time
        thread=01 expected≈10.00% got= 9.99% of on-cpu time
        thread=08 expected≈10.00% got=10.02% of on-cpu time
        thread=05 expected≈10.00% got=10.02% of on-cpu time
    --- PASS: TestEquivalentGoroutines (0.56s)

    === RUN   TestProportionalGoroutines
        thread=01 got  1.82% of on-cpu time: expected≈ 1.00x got=1.00x
        thread=02 got  3.64% of on-cpu time: expected≈ 2.00x got=2.00x
        thread=03 got  5.47% of on-cpu time: expected≈ 3.00x got=3.00x
        thread=04 got  7.28% of on-cpu time: expected≈ 4.00x got=4.00x
        thread=05 got  9.09% of on-cpu time: expected≈ 5.00x got=4.99x
        thread=06 got 10.91% of on-cpu time: expected≈ 6.00x got=5.99x
        thread=07 got 12.73% of on-cpu time: expected≈ 7.00x got=6.99x
        thread=08 got 14.54% of on-cpu time: expected≈ 8.00x got=7.99x
        thread=09 got 16.36% of on-cpu time: expected≈ 9.00x got=8.99x
        thread=10 got 18.16% of on-cpu time: expected≈10.00x got=9.97x
    --- PASS: TestProportionalGoroutines (1.72s)

    === RUN   TestPingPongHog
        pinger/ponger expected≈1.00x got=0.96x
    --- PASS: TestPingPongHog (0.91s)

Release note: None

85987: sql: fix aggregation of statistics r=maryliag a=maryliag

Previously, because we were using a join, we were double
counting statistics when we had the same fingerprint in
memory and persisted.
This commit adds a `DISTINCT` so we only count them once.

Fixes #85958

Release note: None

86008: sql: logic test to inform maximum builtin function oid change r=chengxiong-ruan a=chengxiong-ruan

This commit adds a logic test to let engineers who added a
new builtin function know that the new builtin function is
constructed earlier than some existing builtin functions at
init time.

Release note: None

Co-authored-by: Rafi Shamim <rafi@cockroachlabs.com>
Co-authored-by: Celia La <celia@cockroachlabs.com>
Co-authored-by: Pavel Kalinnikov <pavel@cockroachlabs.com>
Co-authored-by: irfan sharif <irfanmahmoudsharif@gmail.com>
Co-authored-by: Marylia Gutierrez <marylia@cockroachlabs.com>
Co-authored-by: Chengxiong Ruan <chengxiongruan@gmail.com>
  • Loading branch information
7 people committed Aug 12, 2022
7 parents 504b437 + bad5944 + 9ea2a19 + bf99114 + 5b0982e + de478e1 + 41c89ab commit ff3fc7e
Show file tree
Hide file tree
Showing 19 changed files with 793 additions and 96 deletions.
2 changes: 2 additions & 0 deletions .github/CODEOWNERS
Validating CODEOWNERS rules …
Original file line number Diff line number Diff line change
Expand Up @@ -352,6 +352,8 @@
/pkg/util/addr/ @cockroachdb/cli-prs @cockroachdb/obs-inf-prs
/pkg/util/metric/ @cockroachdb/obs-inf-prs
/pkg/util/stop/ @cockroachdb/kv-prs
/pkg/util/grunning/ @cockroachdb/kv-prs
/pkg/util/admission/ @cockroachdb/kv-prs
/pkg/util/tracing @cockroachdb/obs-inf-prs
/pkg/workload/ @cockroachdb/sql-experience-noreview
/pkg/obsservice/ @cockroachdb/obs-inf-prs
Expand Down
4 changes: 4 additions & 0 deletions pkg/BUILD.bazel
Original file line number Diff line number Diff line change
Expand Up @@ -533,6 +533,7 @@ ALL_TESTS = [
"//pkg/util/fuzzystrmatch:fuzzystrmatch_test",
"//pkg/util/goschedstats:goschedstats_test",
"//pkg/util/grpcutil:grpcutil_test",
"//pkg/util/grunning:grunning_test",
"//pkg/util/hlc:hlc_test",
"//pkg/util/httputil:httputil_test",
"//pkg/util/humanizeutil:humanizeutil_test",
Expand Down Expand Up @@ -1870,6 +1871,8 @@ GO_TARGETS = [
"//pkg/util/growstack:growstack",
"//pkg/util/grpcutil:grpcutil",
"//pkg/util/grpcutil:grpcutil_test",
"//pkg/util/grunning:grunning",
"//pkg/util/grunning:grunning_test",
"//pkg/util/hlc:hlc",
"//pkg/util/hlc:hlc_test",
"//pkg/util/httputil:httputil",
Expand Down Expand Up @@ -2827,6 +2830,7 @@ GET_X_DATA_TARGETS = [
"//pkg/util/goschedstats:get_x_data",
"//pkg/util/growstack:get_x_data",
"//pkg/util/grpcutil:get_x_data",
"//pkg/util/grunning:get_x_data",
"//pkg/util/hlc:get_x_data",
"//pkg/util/httputil:get_x_data",
"//pkg/util/humanizeutil:get_x_data",
Expand Down
20 changes: 4 additions & 16 deletions pkg/clusterversion/cockroach_versions.go
Original file line number Diff line number Diff line change
Expand Up @@ -164,11 +164,6 @@ const (
// Start22_1 demarcates work towards CockroachDB v22.1.
Start22_1

// PebbleFormatBlockPropertyCollector switches to a backwards incompatible
// Pebble version that provides block property collectors that can be used
// for fine-grained time bound iteration. See
// https://github.com/cockroachdb/pebble/issues/1190 for details.
PebbleFormatBlockPropertyCollector
// ProbeRequest is the version at which roachpb.ProbeRequest was introduced.
// This version must be active before any ProbeRequest is issued on the
// cluster.
Expand Down Expand Up @@ -244,9 +239,6 @@ const (
EnableDeclarativeSchemaChanger
// RowLevelTTL is the version where we allow row level TTL tables.
RowLevelTTL
// PebbleFormatSplitUserKeysMarked performs a Pebble-level migration and
// upgrades the Pebble format major version to FormatSplitUserKeysMarked.
PebbleFormatSplitUserKeysMarked
// EnableNewStoreRebalancer enables the new store rebalancer introduced in
// 22.1.
EnableNewStoreRebalancer
Expand Down Expand Up @@ -369,6 +361,10 @@ const (
// previously referenced a < 21.2 version until that check/gate can be removed.
const TODOPreV21_2 = V21_2

// TODOPreV22_1 is an alias for V22_1 for use in any version gate/check that
// previously referenced a < 22.1 version until that check/gate can be removed.
const TODOPreV22_1 = V22_1

// versionsSingleton lists all historical versions here in chronological order,
// with comments describing what backwards-incompatible features were
// introduced.
Expand Down Expand Up @@ -398,10 +394,6 @@ var versionsSingleton = keyedVersions{
Key: Start22_1,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 2},
},
{
Key: PebbleFormatBlockPropertyCollector,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 24},
},
{
Key: ProbeRequest,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 26},
Expand Down Expand Up @@ -489,10 +481,6 @@ var versionsSingleton = keyedVersions{
Key: RowLevelTTL,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 88},
},
{
Key: PebbleFormatSplitUserKeysMarked,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 90},
},
{
Key: EnableNewStoreRebalancer,
Version: roachpb.Version{Major: 21, Minor: 2, Internal: 96},
Expand Down
114 changes: 56 additions & 58 deletions pkg/clusterversion/key_string.go

Some generated files are not rendered by default. Learn more about how customized files appear on GitHub.

13 changes: 7 additions & 6 deletions pkg/kv/kvserver/raft_transport.go
Original file line number Diff line number Diff line change
Expand Up @@ -13,6 +13,7 @@ package kvserver
import (
"context"
"net"
"runtime/pprof"
"time"
"unsafe"

Expand Down Expand Up @@ -373,10 +374,7 @@ func (t *RaftTransport) Stop(storeID roachpb.StoreID) {
// lost and a new instance of processQueue will be started by the next message
// to be sent.
func (t *RaftTransport) processQueue(
nodeID roachpb.NodeID,
ch chan *kvserverpb.RaftMessageRequest,
stream MultiRaft_RaftMessageBatchClient,
class rpc.ConnectionClass,
ch chan *kvserverpb.RaftMessageRequest, stream MultiRaft_RaftMessageBatchClient,
) error {
errCh := make(chan error, 1)

Expand Down Expand Up @@ -566,11 +564,14 @@ func (t *RaftTransport) startProcessNewQueue(
return
}

if err := t.processQueue(toNodeID, ch, stream, class); err != nil {
if err := t.processQueue(ch, stream); err != nil {
log.Warningf(ctx, "while processing outgoing Raft queue to node %d: %s:", toNodeID, err)
}
}
err := t.stopper.RunAsyncTask(ctx, "storage.RaftTransport: sending messages", worker)
err := t.stopper.RunAsyncTask(ctx, "storage.RaftTransport: sending/receiving messages",
func(ctx context.Context) {
pprof.Do(ctx, pprof.Labels("remote_node_id", toNodeID.String()), worker)
})
if err != nil {
t.queues[class].Delete(int64(toNodeID))
return false
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/crdb_internal.go
Original file line number Diff line number Diff line change
Expand Up @@ -5571,7 +5571,7 @@ SELECT
plan_hash,
app_name,
max(metadata) as metadata,
crdb_internal.merge_statement_stats(array_agg(statistics)),
crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)),
max(sampled_plan),
aggregation_interval,
array_remove(array_agg(index_rec), NULL) AS index_recommendations
Expand Down
19 changes: 19 additions & 0 deletions pkg/sql/importer/import_stmt_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -702,6 +702,25 @@ ORDER BY table_name
},
},
},
{
name: "array",
create: `a string, b string[]`,
typ: "CSV",
data: `cat,"{somevalue,anothervalue,anothervalue123}"`,
query: map[string][][]string{
`SELECT * from t`: {
{"cat", "{somevalue,anothervalue,anothervalue123}"},
},
},
},
{
name: "array",
create: `a string, b string[]`,
typ: "CSV",
data: `dog,{some,thing}`,
err: "error parsing row 1: expected 2 fields, got 3",
rejected: "dog,{some,thing}\n",
},

// PG COPY
{
Expand Down
17 changes: 12 additions & 5 deletions pkg/sql/importer/read_import_csv.go
Original file line number Diff line number Diff line change
Expand Up @@ -221,11 +221,18 @@ func (c *csvRowConsumer) FillDatums(
var err error
conv.Datums[datumIdx], err = rowenc.ParseDatumStringAs(conv.VisibleColTypes[i], field.Val, conv.EvalCtx)
if err != nil {
col := conv.VisibleCols[i]
return newImportRowError(
errors.Wrapf(err, "parse %q as %s", col.GetName(), col.GetType().SQLString()),
strRecord(record, c.opts.Comma),
rowNum)
// Fallback to parsing as a string literal. This allows us to support
// both array expressions (like `ARRAY[1, 2, 3]`) and literals (like
// `{1, 2, 3}`).
var err2 error
conv.Datums[datumIdx], _, err2 = tree.ParseAndRequireString(conv.VisibleColTypes[i], field.Val, conv.EvalCtx)
if err2 != nil {
col := conv.VisibleCols[i]
return newImportRowError(
errors.Wrapf(errors.CombineErrors(err, err2), "parse %q as %s", col.GetName(), col.GetType().SQLString()),
strRecord(record, c.opts.Comma),
rowNum)
}
}
}
datumIdx++
Expand Down
4 changes: 2 additions & 2 deletions pkg/sql/logictest/testdata/logic_test/create_statements
Original file line number Diff line number Diff line change
Expand Up @@ -1512,7 +1512,7 @@ CREATE VIEW crdb_internal.statement_statistics (
plan_hash,
app_name,
max(metadata) AS metadata,
crdb_internal.merge_statement_stats(array_agg(statistics)),
crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)),
max(sampled_plan),
aggregation_interval,
array_remove(array_agg(index_rec), NULL) AS index_recommendations
Expand Down Expand Up @@ -1571,7 +1571,7 @@ CREATE VIEW crdb_internal.statement_statistics (
plan_hash,
app_name,
max(metadata) AS metadata,
crdb_internal.merge_statement_stats(array_agg(statistics)),
crdb_internal.merge_statement_stats(array_agg(DISTINCT statistics)),
max(sampled_plan),
aggregation_interval,
array_remove(array_agg(index_rec), NULL) AS index_recommendations
Expand Down
15 changes: 15 additions & 0 deletions pkg/sql/logictest/testdata/logic_test/pg_catalog
Original file line number Diff line number Diff line change
Expand Up @@ -4670,6 +4670,21 @@ query T
SELECT proname FROM pg_catalog.pg_proc WHERE oid = 0
----

let $cur_max_builtin_oid
SELECT cur_max_builtin_oid FROM [SELECT max(oid) as cur_max_builtin_oid FROM pg_catalog.pg_proc]

## If this test failed (proname is the same, but oid increased), it's likely that a
## new builtin function is implemented and it's somewhere in the middle of the
## existing functions at init time. Though the changes to builtin function OID is
## generally ok, it's still better if we could move the new implement to end of the
## list at init time (see all_builtins.go)
## TODO(chengxiong): consider to have a deterministic list of builtin function oids
## so that new implementations can just be added to it.
query TT
SELECT proname, oid FROM pg_catalog.pg_proc WHERE oid = $cur_max_builtin_oid
----
to_regtype 2031

## Ensure that unnest works with oid wrapper arrays

query O
Expand Down
Loading

0 comments on commit ff3fc7e

Please sign in to comment.