Skip to content

Commit

Permalink
Merge #82622 #82632
Browse files Browse the repository at this point in the history
82622: sql: don't rewind to BEGIN when retrying txn r=ZhouXing19 a=rafiss

fixes #82392

Release note (bug fix): Fixed a bug where CockroachDB would
sometimes automatically retry the BEGIN statement of an explicit
transaction.

82632: sql: fix SPLIT AT and EXPERIMENTAL_RELOCATE for multi-column geo-spatial indexes r=mgartner a=mgartner

#### sql: add index hints for inverted_filter_geospatial_dist

This commit adds index hints to queries in
`invert_filter_geospatial_dist` logic tests. This ensures that the tests
are testing inverted filterer logic, which is their intent.

Release note: None

#### sql: fix SPLIT AT and EXPERIMENTAL_RELOCATE for multi-column geo-spatial indexes

This commit fixes a bug that prevented `ALTER INDEX ... SPLIT AT` and
`ALTER TABLE ... EXPERIMENTAL_RELOCATE` from working on multi-column
geo-spatial indexes.

Release note: None


Co-authored-by: Rafi Shamim <[email protected]>
Co-authored-by: Marcus Gartner <[email protected]>
  • Loading branch information
3 people committed Jun 9, 2022
3 parents 4d5966e + 32c4985 + 0bd0b72 commit b6f42c2
Show file tree
Hide file tree
Showing 7 changed files with 144 additions and 25 deletions.
6 changes: 4 additions & 2 deletions pkg/sql/conn_executor.go
Original file line number Diff line number Diff line change
Expand Up @@ -2181,7 +2181,9 @@ func (ex *connExecutor) updateTxnRewindPosMaybe(
if _, ok := ex.machine.CurState().(stateOpen); !ok {
return nil
}
if advInfo.txnEvent.eventType == txnStart || advInfo.txnEvent.eventType == txnRestart {
if advInfo.txnEvent.eventType == txnStart ||
advInfo.txnEvent.eventType == txnRestart ||
advInfo.txnEvent.eventType == txnUpgradeToExplicit {
var nextPos CmdPos
switch advInfo.code {
case stayInPlace:
Expand Down Expand Up @@ -2859,7 +2861,7 @@ func (ex *connExecutor) txnStateTransitionsApplyWrapper(

// Handle transaction events which cause updates to txnState.
switch advInfo.txnEvent.eventType {
case noEvent:
case noEvent, txnUpgradeToExplicit:
_, nextStateIsAborted := ex.machine.CurState().(stateAborted)
// Update the deadline on the transaction based on the collections,
// if the transaction is currently open. If the next state is aborted
Expand Down
52 changes: 52 additions & 0 deletions pkg/sql/conn_executor_test.go
Original file line number Diff line number Diff line change
Expand Up @@ -727,6 +727,58 @@ func TestRetriableErrorDuringPrepare(t *testing.T) {
defer func() { _ = stmt.Close() }()
}

// TestRetriableErrorDuringUpgradedTransaction ensures that a retriable error
// that happens during a transaction that was upgraded from an implicit
// transaction into an explicit transaction does not cause the BEGIN to be
// re-executed.
func TestRetriableErrorDuringUpgradedTransaction(t *testing.T) {
defer leaktest.AfterTest(t)()
defer log.Scope(t).Close(t)
var retryCount int64
const numToRetry = 2 // only fail on the first two attempts
filter := newDynamicRequestFilter()
s, sqlDB, _ := serverutils.StartServer(t, base.TestServerArgs{
Knobs: base.TestingKnobs{
Store: &kvserver.StoreTestingKnobs{
TestingRequestFilter: filter.filter,
},
},
})
defer s.Stopper().Stop(context.Background())

conn, err := sqlDB.Conn(context.Background())
require.NoError(t, err)
testDB := sqlutils.MakeSQLRunner(conn)

var fooTableId uint32
testDB.Exec(t, "SET enable_implicit_transaction_for_batch_statements = true")
testDB.Exec(t, "CREATE TABLE foo (a INT PRIMARY KEY)")
testDB.QueryRow(t, "SELECT 'foo'::regclass::oid").Scan(&fooTableId)

// Inject an error that will happen during execution.
filter.setFilter(func(ctx context.Context, ba roachpb.BatchRequest) *roachpb.Error {
if ba.Txn == nil {
return nil
}
if req, ok := ba.GetArg(roachpb.ConditionalPut); ok {
put := req.(*roachpb.ConditionalPutRequest)
_, tableID, err := keys.SystemSQLCodec.DecodeTablePrefix(put.Key)
if err != nil || tableID != fooTableId {
return nil
}
if atomic.AddInt64(&retryCount, 1) <= numToRetry {
return roachpb.NewErrorWithTxn(
roachpb.NewTransactionRetryError(roachpb.RETRY_REASON_UNKNOWN, "injected retry error"), ba.Txn,
)
}
}
return nil
})

testDB.Exec(t, "SELECT 1; BEGIN; INSERT INTO foo VALUES(1); COMMIT;")
require.Equal(t, numToRetry+1, int(retryCount))
}

// This test ensures that when in an explicit transaction and statement
// preparation uses the user's transaction, errors during those planning queries
// are handled correctly.
Expand Down
2 changes: 1 addition & 1 deletion pkg/sql/conn_fsm.go
Original file line number Diff line number Diff line change
Expand Up @@ -303,7 +303,7 @@ var TxnStateTransitions = fsm.Compile(fsm.Pattern{
args.Extended.(*txnState).setAdvanceInfo(
advanceOne,
noRewind,
txnEvent{eventType: noEvent},
txnEvent{eventType: txnUpgradeToExplicit},
)
return nil
},
Expand Down
Original file line number Diff line number Diff line change
@@ -1,32 +1,33 @@
# LogicTest: 5node

statement ok
CREATE TABLE geo_table(
k int primary key,
geom geometry,
INVERTED INDEX geom_index(geom)
CREATE TABLE geo_table (
k INT PRIMARY KEY,
s STRING,
geom GEOMETRY,
INVERTED INDEX geom_index (geom)
)

statement ok
INSERT INTO geo_table VALUES
(1, 'POINT(1 1)'),
(2, 'LINESTRING(1 1, 2 2)'),
(3, 'POINT(3 3)'),
(4, 'LINESTRING(4 4, 5 5)'),
(5, 'LINESTRING(40 40, 41 41)'),
(6, 'POLYGON((1 1, 5 1, 5 5, 1 5, 1 1))'),
(7, 'LINESTRING(1 1, 3 3)')
(1, 'foo', 'POINT(1 1)'),
(2, 'foo', 'LINESTRING(1 1, 2 2)'),
(3, 'foo', 'POINT(3 3)'),
(4, 'bar', 'LINESTRING(4 4, 5 5)'),
(5, 'bar', 'LINESTRING(40 40, 41 41)'),
(6, 'bar', 'POLYGON((1 1, 5 1, 5 5, 1 5, 1 1))'),
(7, 'foo', 'LINESTRING(1 1, 3 3)')

# Not distributed.
query I
SELECT k FROM geo_table WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
SELECT k FROM geo_table@geom_index WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
3
6
7

query I
SELECT k FROM geo_table WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
SELECT k FROM geo_table@geom_index WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
6
7
Expand Down Expand Up @@ -66,15 +67,15 @@ NULL /1152921574000000000 {1} 1

# Distributed.
query I
SELECT k FROM geo_table WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
SELECT k FROM geo_table@geom_index WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
3
6
7

# Data is distributed, but the filterer can't be distributed since it is not a union.
query I
SELECT k FROM geo_table WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
SELECT k FROM geo_table@geom_index WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
6
7
Expand All @@ -93,15 +94,73 @@ NULL /1152921574000000000 {2} 2
/1152921574000000000 NULL {2} 2

query I
SELECT k FROM geo_table WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
SELECT k FROM geo_table@geom_index WHERE ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
3
6
7

# Filtering is placed at node 2.
query I
SELECT k FROM geo_table WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
SELECT k FROM geo_table@geom_index WHERE ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
6
7

statement ok
DROP INDEX geom_index;

# Test for multi-column geo-spatial indexes.
statement ok
CREATE INVERTED INDEX geom_index2 ON geo_table(s, geom)

query TI colnames,rowsort
SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2]
----
replicas lease_holder
{2} 2

# Not distributed.
query I
SELECT k FROM geo_table@geom_index2 WHERE s = 'foo' AND ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
3
7

query I
SELECT k FROM geo_table@geom_index2 WHERE s = 'foo' AND ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
7

statement ok
ALTER INDEX geo_table@geom_index2 SPLIT AT VALUES ('foo', 1152921574000000000)

query TI colnames,rowsort
SELECT replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2]
----
replicas lease_holder
{2} 2
{2} 2

statement ok
ALTER INDEX geo_table@geom_index2 EXPERIMENTAL_RELOCATE VALUES (ARRAY[1], 'foo', 1152921574000000000)

query TTTI colnames,rowsort
SELECT start_key, end_key, replicas, lease_holder FROM [SHOW RANGES FROM INDEX geo_table@geom_index2]
----
start_key end_key replicas lease_holder
NULL /"foo"/1152921574000000000 {2} 2
/"foo"/1152921574000000000 NULL {1} 1

# Distributed.
query I
SELECT k FROM geo_table@geom_index2 WHERE s = 'foo' AND ST_Intersects('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
3
7

# Data is distributed, but the filterer can't be distributed since it is not a union.
query I
SELECT k FROM geo_table@geom_index2 WHERE s = 'foo' AND ST_CoveredBy('MULTIPOINT((2.2 2.2), (3.0 3.0))'::geometry, geom) ORDER BY k
----
7
6 changes: 3 additions & 3 deletions pkg/sql/opt/optbuilder/alter_table.go
Original file line number Diff line number Diff line change
Expand Up @@ -206,9 +206,9 @@ func getIndexColumnNamesAndTypes(index cat.Index) (colNames []string, colTypes [
// TODO(sumeer): special case Array too. JSON is harder since the split
// needs to be a Datum and the JSON inverted column is not.
//
// Geospatial inverted index. The first column is the inverted column and
// is an int.
colTypes[0] = types.Int
// Geospatial inverted index. The last explicit column is the inverted
// column and is an int.
colTypes[index.ExplicitColumnCount()-1] = types.Int
}
return colNames, colTypes
}
Expand Down
5 changes: 5 additions & 0 deletions pkg/sql/txn_state.go
Original file line number Diff line number Diff line change
Expand Up @@ -402,6 +402,11 @@ const (
// the transaction. This allows such savepoints to reset more state than other
// savepoints.
txnRestart
// txnUpgradeToExplicit means that the current implicit transaction was
// upgraded to an explicit one. This happens when BEGIN is executed during the
// extended protocol or as part of a batch of statements. It's used to
// indicate that the transaction rewind position should be updated.
txnUpgradeToExplicit
)

// advanceInfo represents instructions for the connExecutor about what statement
Expand Down
5 changes: 3 additions & 2 deletions pkg/sql/txneventtype_string.go

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

0 comments on commit b6f42c2

Please sign in to comment.