@@ -121,72 +121,103 @@ func TestExternalRowDataDistSQL(t *testing.T) {
121121 ctx := context .Background ()
122122
123123 // Start a 5-node cluster.
124- tc := serverutils .StartCluster (t , 5 , /* numNodes */
124+ testCluster := serverutils .StartCluster (t , 5 , /* numNodes */
125125 base.TestClusterArgs {
126126 ReplicationMode : base .ReplicationManual ,
127127 ServerArgs : base.TestServerArgs {
128- UseDatabase : "defaultdb" ,
128+ UseDatabase : "defaultdb" ,
129+ DefaultTestTenant : base .TestControlsTenantsExplicitly ,
129130 },
130131 })
131- defer tc .Stopper ().Stop (ctx )
132-
133- r0 := sqlutils .MakeSQLRunner (tc .ApplicationLayer (0 ).SQLConn (t ))
134- r0 .Exec (t , `CREATE TABLE t (k INT PRIMARY KEY, v1 INT, v2 INT)` )
135- r0 .Exec (t , `CREATE TABLE t_copy (k INT PRIMARY KEY, v1 INT, v2 INT)` )
136- r0 .Exec (t , `INSERT INTO t VALUES (1), (3), (5)` )
137-
138- if tc .StartedDefaultTestTenant () {
139- // Grant capability to run RELOCATE to secondary (test) tenant.
140- systemDB := sqlutils .MakeSQLRunner (tc .SystemLayer (0 ).SQLConn (t ))
141- systemDB .Exec (t ,
142- `ALTER TENANT [$1] GRANT CAPABILITY can_admin_relocate_range=true` ,
143- serverutils .TestTenantID ().ToUint64 ())
144- }
145-
146- // Place leaseholders on nodes 3, 4, 5.
147- r0 .Exec (t , `ALTER TABLE t SPLIT AT VALUES (2), (4)` )
148- r0 .ExecSucceedsSoon (
149- t , `ALTER TABLE t RELOCATE VALUES (ARRAY[3], 1), (ARRAY[4], 3), (ARRAY[5], 5)` ,
132+ defer testCluster .Stopper ().Stop (ctx )
133+ ts := testCluster .Server (0 )
134+
135+ srcTenant , srcDB , err := ts .TenantController ().StartSharedProcessTenant (ctx ,
136+ base.TestSharedProcessTenantArgs {
137+ TenantID : serverutils .TestTenantID (),
138+ TenantName : "src" ,
139+ UseDatabase : "defaultdb" ,
140+ },
141+ )
142+ require .NoError (t , err )
143+ dstTenant , dstDB , err := ts .TenantController ().StartSharedProcessTenant (ctx ,
144+ base.TestSharedProcessTenantArgs {
145+ TenantID : serverutils .TestTenantID2 (),
146+ TenantName : "dst" ,
147+ UseDatabase : "defaultdb" ,
148+ },
149+ )
150+ require .NoError (t , err )
151+
152+ // Set up the source table.
153+ srcRunner := sqlutils .MakeSQLRunner (srcDB )
154+ otherRunner := sqlutils .MakeSQLRunner (dstDB )
155+
156+ // Set up the source table. Place leaseholders on nodes 3, 4, 5.
157+ srcRunner .Exec (t , `CREATE TABLE t_src (k INT PRIMARY KEY, v1 INT, v2 INT)` )
158+ srcRunner .Exec (t , `INSERT INTO t_src VALUES (1), (3), (5)` )
159+ srcRunner .Exec (t , `ALTER TABLE t_src SPLIT AT VALUES (2), (4)` )
160+ srcRunner .ExecSucceedsSoon (
161+ t , `ALTER TABLE t_src RELOCATE VALUES (ARRAY[3], 1), (ARRAY[4], 3), (ARRAY[5], 5)` ,
150162 )
151163
152- asOf := hlc.Timestamp {WallTime : timeutil .Now ().UnixNano ()}
153-
154- // Modify the table descriptor for 't_copy' to have external row data from
155- // table 't'.
156- var tableID int
157- row := r0 .QueryRow (t , `SELECT 't'::REGCLASS::OID` )
158- row .Scan (& tableID )
159- execCfg0 := tc .ApplicationLayer (0 ).ExecutorConfig ().(sql.ExecutorConfig )
160- require .NoError (t , execCfg0 .InternalDB .DescsTxn (ctx , func (ctx context.Context , txn descs.Txn ) error {
161- descriptors := txn .Descriptors ()
162- tn := tree .MakeTableNameWithSchema ("defaultdb" , "public" , "t_copy" )
163- _ , mut , err := descs .PrefixAndMutableTable (ctx , descriptors .MutableByName (txn .KV ()), & tn )
164- if err != nil {
165- return err
166- }
167- require .NotNil (t , mut )
168- mut .External = & descpb.ExternalRowData {
169- AsOf : asOf ,
170- TenantID : execCfg0 .Codec .TenantID ,
171- TableID : descpb .ID (tableID ),
172- }
173- return descriptors .WriteDesc (ctx , false /* kvtrace */ , mut , txn .KV ())
174- }))
175-
176- // Now check that DistSQL plans against both tables correctly place
177- // flows on nodes 1, 3, 4, 5.
178- r0 .Exec (t , `SET distsql = always` )
179-
180- exp := `"nodeNames":["1","3","4","5"]`
181- var info string
182- row = r0 .QueryRow (t , `EXPLAIN (DISTSQL, JSON) SELECT count(*) FROM t` )
183- row .Scan (& info )
184- if ! strings .Contains (info , exp ) {
185- t .Fatalf ("expected DistSQL plan to contain %s: was %s" , exp , info )
186- }
187- row = r0 .QueryRow (t , `EXPLAIN (DISTSQL, JSON) SELECT count(*) FROM t_copy` )
188- row .Scan (& info )
189- if ! strings .Contains (info , exp ) {
190- t .Fatalf ("expected DistSQL plan to contain %s: was %s" , exp , info )
164+ for _ , tc := range []struct {
165+ name string
166+ dstRunner * sqlutils.SQLRunner
167+ dstInternalDB * sql.InternalDB
168+ }{
169+ {
170+ name : "same-tenant" ,
171+ dstRunner : srcRunner ,
172+ dstInternalDB : srcTenant .InternalDB ().(* sql.InternalDB ),
173+ },
174+ {
175+ name : "different-tenant" ,
176+ dstRunner : otherRunner ,
177+ dstInternalDB : dstTenant .InternalDB ().(* sql.InternalDB ),
178+ },
179+ } {
180+ t .Run (tc .name , func (t * testing.T ) {
181+ tc .dstRunner .Exec (t , `CREATE TABLE t_dst (k INT PRIMARY KEY, v1 INT, v2 INT)` )
182+ asOf := hlc.Timestamp {WallTime : timeutil .Now ().UnixNano ()}
183+
184+ // Modify the table descriptor for 't_dst' to have external row data from
185+ // table 't_src'.
186+ var tableID int
187+ row := srcRunner .QueryRow (t , `SELECT 't_src'::REGCLASS::OID` )
188+ row .Scan (& tableID )
189+ require .NoError (t , tc .dstInternalDB .DescsTxn (ctx , func (ctx context.Context , txn descs.Txn ) error {
190+ descriptors := txn .Descriptors ()
191+ tn := tree .MakeTableNameWithSchema ("defaultdb" , "public" , "t_dst" )
192+ _ , mut , err := descs .PrefixAndMutableTable (ctx , descriptors .MutableByName (txn .KV ()), & tn )
193+ if err != nil {
194+ return err
195+ }
196+ require .NotNil (t , mut )
197+ mut .External = & descpb.ExternalRowData {
198+ AsOf : asOf ,
199+ TenantID : serverutils .TestTenantID (),
200+ TableID : descpb .ID (tableID ),
201+ }
202+ return descriptors .WriteDesc (ctx , false /* kvtrace */ , mut , txn .KV ())
203+ }))
204+
205+ // Now check that DistSQL plans against both tables correctly place
206+ // flows on nodes 1, 3, 4, 5.
207+ srcRunner .Exec (t , `SET distsql = always` )
208+ tc .dstRunner .Exec (t , `SET distsql = always` )
209+ exp := `"nodeNames":["1","3","4","5"]`
210+ var info string
211+ row = srcRunner .QueryRow (t , `EXPLAIN (DISTSQL, JSON) SELECT count(*) FROM t_src` )
212+ row .Scan (& info )
213+ if ! strings .Contains (info , exp ) {
214+ t .Fatalf ("expected DistSQL plan to contain %s: was %s" , exp , info )
215+ }
216+ row = tc .dstRunner .QueryRow (t , `EXPLAIN (DISTSQL, JSON) SELECT count(*) FROM t_dst` )
217+ row .Scan (& info )
218+ if ! strings .Contains (info , exp ) {
219+ t .Fatalf ("expected DistSQL plan to contain %s: was %s" , exp , info )
220+ }
221+ })
191222 }
192223}
0 commit comments