Skip to content

Commit

Permalink
Browse files Browse the repository at this point in the history
82691: storage: add range key support for `MVCCIncrementalIterator` r=erikgrinaker a=erikgrinaker

**storage: minor `MVCCIncrementalIterator` cleanup**

This patch makes some minor, mechanical cleanups of
`MVCCIncrementalIterator`:

* `checkValidAndSaveErr` was renamed `updateValid`.

* `initMetaAndCheckForIntentOrInlineError` was renamed `updateMeta`.

* `NextKeyIgnoringTime` was removed, as it was never used.

* `Key()` and `Value()` were removed, as they are not part of the
  `SimpleMVCCIterator` interface.

* Method comments that were copies of interface method comments were
  replaced with "implements SimpleMVCCIterator" variants.

Release note: None

**storage: add range key support for `MVCCIncrementalIterator`**

This patch adds range key support for `MVCCIncrementalIterator`,
filtering them by the time bounds and exposing them via the usual
`SimpleMVCCIterator` interface.

This comes with a moderate performance penalty in the no-range-key case,
mostly due to additional `HasPointAndRange()` checks. This, and the
range key paths, will be optimized later. For now, correctness is
sufficient, in order to unblock higher-level work.

```
name                                                 old time/op  new time/op  delta
MVCCIncrementalIterator/ts=5-24                      11.7ms ± 9%  12.1ms ± 3%    ~     (p=0.065 n=9+10)
MVCCIncrementalIterator/ts=480-24                     442µs ± 1%   444µs ± 1%    ~     (p=0.094 n=9+9)
MVCCIncrementalIteratorForOldData/valueSize=100-24   1.41ms ± 1%  1.49ms ± 1%  +5.59%  (p=0.000 n=10+10)
MVCCIncrementalIteratorForOldData/valueSize=500-24   1.89ms ± 2%  1.98ms ± 2%  +4.61%  (p=0.000 n=10+10)
MVCCIncrementalIteratorForOldData/valueSize=1000-24  2.59ms ± 2%  2.68ms ± 1%  +3.33%  (p=0.000 n=10+10)
MVCCIncrementalIteratorForOldData/valueSize=2000-24  4.15ms ± 2%  4.12ms ± 3%    ~     (p=0.481 n=10+10)
```

Release note: None

82754: sql: fix DECLARE with pgwire placeholders r=jordanlewis a=jordanlewis

Closes cockroachdb#82565
Closes cockroachdb#77067

Previously, it was not possible to prepare a DECLARE statement via
pgwire if it included placeholders. That limitation is now lifted.

cc `@dvarrazzo`

Release note (bug fix): add missing support for preparing a DECLARE
cursor statement with placeholders.

Co-authored-by: Erik Grinaker <[email protected]>
Co-authored-by: Jordan Lewis <[email protected]>
  • Loading branch information
3 people committed Jun 16, 2022
3 parents 277921b + 544393e + b7c45c1 commit 877542d
Show file tree
Hide file tree
Showing 10 changed files with 1,961 additions and 370 deletions.
2 changes: 1 addition & 1 deletion pkg/kv/kvserver/batcheval/cmd_refresh_range.go
Original file line number Diff line number Diff line change
Expand Up @@ -89,7 +89,7 @@ func refreshRange(
break
}

key := iter.Key()
key := iter.UnsafeKey().Clone()
if !key.IsValue() {
// Found an intent. Check whether it is owned by this transaction.
// If so, proceed with iteration. Otherwise, return an error.
Expand Down
38 changes: 38 additions & 0 deletions pkg/sql/pgwire/testdata/pgtest/portals
Original file line number Diff line number Diff line change
Expand Up @@ -1356,3 +1356,41 @@ ReadyForQuery
----
{"Type":"RowDescription","Fields":[{"Name":"g","TableOID":0,"TableAttributeNumber":0,"DataTypeOID":20,"DataTypeSize":8,"TypeModifier":-1,"Format":0}]}
{"Type":"ReadyForQuery","TxStatus":"T"}

# Test preparing a SQL-level DECLARE with placeholders

send
Parse {"Query": "DECLARE bar CURSOR FOR SELECT g::INT8 FROM generate_series(1, $1) g(g)", "ParameterOIDs": [21]}
Bind {"PreparedStatement": "", "ParameterFormatCodes": [0], "ResultFormatCodes": [0], "Parameters": [{"text":"2"}]}
Describe {"ObjectType": "P", "Name": ""}
Execute
Sync
----

until
ReadyForQuery
----
{"Type":"ParseComplete"}
{"Type":"BindComplete"}
{"Type":"NoData"}
{"Type":"CommandComplete","CommandTag":"DECLARE CURSOR"}
{"Type":"ReadyForQuery","TxStatus":"T"}

send
Parse {"Query": "FETCH 2 bar"}
Bind
Describe {"ObjectType": "P", "Name": ""}
Execute
Sync
----

until
ReadyForQuery
----
{"Type":"ParseComplete"}
{"Type":"BindComplete"}
{"Type":"RowDescription","Fields":[{"Name":"g","TableOID":0,"TableAttributeNumber":0,"DataTypeOID":20,"DataTypeSize":8,"TypeModifier":-1,"Format":0}]}
{"Type":"DataRow","Values":[{"text":"1"}]}
{"Type":"DataRow","Values":[{"text":"2"}]}
{"Type":"CommandComplete","CommandTag":"FETCH 2"}
{"Type":"ReadyForQuery","TxStatus":"T"}
10 changes: 10 additions & 0 deletions pkg/sql/plan_opt.go
Original file line number Diff line number Diff line change
Expand Up @@ -108,6 +108,16 @@ func (p *planner) prepareUsingOptimizer(ctx context.Context) (planFlags, error)
}
stmt.Prepared.Columns = colinfo.ExplainPlanColumns
return opc.flags, nil
case *tree.DeclareCursor:
// Build memo for the purposes of typing placeholders.
// TODO(jordan): converting DeclareCursor to not be an opaque statement
// would be a better way to accomplish this goal. See CREATE TABLE for an
// example.
f := opc.optimizer.Factory()
bld := optbuilder.New(ctx, &p.semaCtx, p.EvalContext(), &opc.catalog, f, t.Select)
if err := bld.Build(); err != nil {
return opc.flags, err
}
}

if opc.useCache {
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/sql_cursor.go
Original file line number Diff line number Diff line change
Expand Up @@ -86,7 +86,7 @@ func (p *planner) DeclareCursor(ctx context.Context, s *tree.DeclareCursor) (pla
"DECLARE CURSOR must not contain data-modifying statements in WITH")
}

statement := s.Select.String()
statement := formatWithPlaceholders(s.Select, p.EvalContext())
itCtx := context.Background()
rows, err := ie.QueryIterator(itCtx, "sql-cursor", p.txn, statement)
if err != nil {
Expand Down
12 changes: 4 additions & 8 deletions pkg/sql/sql_cursor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -16,7 +16,6 @@ import (

"github.com/cockroachdb/cockroach/pkg/base"
"github.com/cockroachdb/cockroach/pkg/testutils/serverutils"
"github.com/cockroachdb/cockroach/pkg/testutils/skip"
"github.com/cockroachdb/cockroach/pkg/util/leaktest"
"github.com/cockroachdb/cockroach/pkg/util/log"
"github.com/stretchr/testify/require"
Expand Down Expand Up @@ -110,15 +109,12 @@ func TestPrepareCursors(t *testing.T) {

// Make sure that we can use the automatic prepare support (when sending
// placeholders) to do the same thing.
// TODO (jordan): This currently doesn't work, because we don't fully walk
// the tree typechecking expressions when sending a DECLARE through the
// optimizer. See issue #77067.
// When this limitation is lifted, the rest of this test should be uncommented.
t.Run("prepare_declare_placeholder", func(t *testing.T) {
_, err = conn.ExecContext(ctx, "DECLARE foo CURSOR FOR SELECT 1 WHERE $1", true)
require.Contains(t, err.Error(), "could not determine data type of placeholder")
_, err = conn.ExecContext(ctx, "BEGIN TRANSACTION")
require.NoError(t, err)

skip.WithIssue(t, 77067, "placeholders in DECLARE not supported")
_, err = conn.ExecContext(ctx, "DECLARE foo CURSOR FOR SELECT 1 WHERE $1", true)
require.NoError(t, err)

stmt, err := conn.PrepareContext(ctx, "FETCH 1 foo")
require.NoError(t, err)
Expand Down
Loading

0 comments on commit 877542d

Please sign in to comment.