github.com/cockroachdb/cockroach@v20.2.0-alpha.1+incompatible/pkg/ccl/changefeedccl/sink_test.go (about) 1 // Copyright 2018 The Cockroach Authors. 2 // 3 // Licensed as a CockroachDB Enterprise file under the Cockroach Community 4 // License (the "License"); you may not use this file except in compliance with 5 // the License. You may obtain a copy of the License at 6 // 7 // https://github.com/cockroachdb/cockroach/blob/master/licenses/CCL.txt 8 9 package changefeedccl 10 11 import ( 12 "context" 13 "net/url" 14 "strconv" 15 "testing" 16 "time" 17 18 "github.com/Shopify/sarama" 19 "github.com/cockroachdb/cockroach/pkg/base" 20 "github.com/cockroachdb/cockroach/pkg/jobs/jobspb" 21 "github.com/cockroachdb/cockroach/pkg/security" 22 "github.com/cockroachdb/cockroach/pkg/sql/sqlbase" 23 "github.com/cockroachdb/cockroach/pkg/testutils" 24 "github.com/cockroachdb/cockroach/pkg/testutils/serverutils" 25 "github.com/cockroachdb/cockroach/pkg/testutils/sqlutils" 26 "github.com/cockroachdb/cockroach/pkg/util/hlc" 27 "github.com/cockroachdb/cockroach/pkg/util/leaktest" 28 "github.com/cockroachdb/errors" 29 "github.com/stretchr/testify/require" 30 ) 31 32 var zeroTS hlc.Timestamp 33 34 type asyncProducerMock struct { 35 inputCh chan *sarama.ProducerMessage 36 successesCh chan *sarama.ProducerMessage 37 errorsCh chan *sarama.ProducerError 38 } 39 40 func (p asyncProducerMock) Input() chan<- *sarama.ProducerMessage { return p.inputCh } 41 func (p asyncProducerMock) Successes() <-chan *sarama.ProducerMessage { return p.successesCh } 42 func (p asyncProducerMock) Errors() <-chan *sarama.ProducerError { return p.errorsCh } 43 func (p asyncProducerMock) AsyncClose() { panic(`unimplemented`) } 44 func (p asyncProducerMock) Close() error { 45 close(p.inputCh) 46 close(p.successesCh) 47 close(p.errorsCh) 48 return nil 49 } 50 51 func TestKafkaSink(t *testing.T) { 52 defer leaktest.AfterTest(t)() 53 54 table := func(name string) *sqlbase.TableDescriptor { 55 return &sqlbase.TableDescriptor{Name: name} 56 } 57 58 ctx := context.Background() 59 p := asyncProducerMock{ 60 inputCh: make(chan *sarama.ProducerMessage, 1), 61 successesCh: make(chan *sarama.ProducerMessage, 1), 62 errorsCh: make(chan *sarama.ProducerError, 1), 63 } 64 sink := &kafkaSink{ 65 producer: p, 66 topics: map[string]struct{}{`t`: {}}, 67 } 68 sink.start() 69 defer func() { 70 if err := sink.Close(); err != nil { 71 t.Fatal(err) 72 } 73 }() 74 75 // No inflight 76 if err := sink.Flush(ctx); err != nil { 77 t.Fatal(err) 78 } 79 80 // Timeout 81 if err := sink.EmitRow(ctx, table(`t`), []byte(`1`), nil, zeroTS); err != nil { 82 t.Fatal(err) 83 } 84 m1 := <-p.inputCh 85 for i := 0; i < 2; i++ { 86 timeoutCtx, cancel := context.WithTimeout(ctx, time.Millisecond) 87 defer cancel() 88 if err := sink.Flush(timeoutCtx); !testutils.IsError( 89 err, `context deadline exceeded`, 90 ) { 91 t.Fatalf(`expected "context deadline exceeded" error got: %+v`, err) 92 } 93 } 94 go func() { p.successesCh <- m1 }() 95 if err := sink.Flush(ctx); err != nil { 96 t.Fatal(err) 97 } 98 99 // Check no inflight again now that we've sent something 100 if err := sink.Flush(ctx); err != nil { 101 t.Fatal(err) 102 } 103 104 // Mixed success and error. 105 if err := sink.EmitRow(ctx, table(`t`), []byte(`2`), nil, zeroTS); err != nil { 106 t.Fatal(err) 107 } 108 m2 := <-p.inputCh 109 if err := sink.EmitRow(ctx, table(`t`), []byte(`3`), nil, zeroTS); err != nil { 110 t.Fatal(err) 111 } 112 m3 := <-p.inputCh 113 if err := sink.EmitRow(ctx, table(`t`), []byte(`4`), nil, zeroTS); err != nil { 114 t.Fatal(err) 115 } 116 m4 := <-p.inputCh 117 go func() { p.successesCh <- m2 }() 118 go func() { 119 p.errorsCh <- &sarama.ProducerError{ 120 Msg: m3, 121 Err: errors.New("m3"), 122 } 123 }() 124 go func() { p.successesCh <- m4 }() 125 if err := sink.Flush(ctx); !testutils.IsError(err, `m3`) { 126 t.Fatalf(`expected "m3" error got: %+v`, err) 127 } 128 129 // Check simple success again after error 130 if err := sink.EmitRow(ctx, table(`t`), []byte(`5`), nil, zeroTS); err != nil { 131 t.Fatal(err) 132 } 133 m5 := <-p.inputCh 134 go func() { p.successesCh <- m5 }() 135 if err := sink.Flush(ctx); err != nil { 136 t.Fatal(err) 137 } 138 } 139 140 func TestKafkaSinkEscaping(t *testing.T) { 141 defer leaktest.AfterTest(t)() 142 143 table := func(name string) *sqlbase.TableDescriptor { 144 return &sqlbase.TableDescriptor{Name: name} 145 } 146 147 ctx := context.Background() 148 p := asyncProducerMock{ 149 inputCh: make(chan *sarama.ProducerMessage, 1), 150 successesCh: make(chan *sarama.ProducerMessage, 1), 151 errorsCh: make(chan *sarama.ProducerError, 1), 152 } 153 sink := &kafkaSink{ 154 producer: p, 155 topics: map[string]struct{}{SQLNameToKafkaName(`☃`): {}}, 156 } 157 sink.start() 158 defer func() { require.NoError(t, sink.Close()) }() 159 if err := sink.EmitRow(ctx, table(`☃`), []byte(`k☃`), []byte(`v☃`), zeroTS); err != nil { 160 t.Fatal(err) 161 } 162 m := <-p.inputCh 163 require.Equal(t, `_u2603_`, m.Topic) 164 require.Equal(t, sarama.ByteEncoder(`k☃`), m.Key) 165 require.Equal(t, sarama.ByteEncoder(`v☃`), m.Value) 166 } 167 168 type testEncoder struct{} 169 170 func (testEncoder) EncodeKey(context.Context, encodeRow) ([]byte, error) { panic(`unimplemented`) } 171 func (testEncoder) EncodeValue(context.Context, encodeRow) ([]byte, error) { panic(`unimplemented`) } 172 func (testEncoder) EncodeResolvedTimestamp( 173 _ context.Context, _ string, ts hlc.Timestamp, 174 ) ([]byte, error) { 175 return []byte(ts.String()), nil 176 } 177 178 func TestSQLSink(t *testing.T) { 179 defer leaktest.AfterTest(t)() 180 181 table := func(name string) *sqlbase.TableDescriptor { 182 return &sqlbase.TableDescriptor{Name: name} 183 } 184 185 ctx := context.Background() 186 s, sqlDBRaw, _ := serverutils.StartServer(t, base.TestServerArgs{UseDatabase: "d"}) 187 defer s.Stopper().Stop(ctx) 188 sqlDB := sqlutils.MakeSQLRunner(sqlDBRaw) 189 sqlDB.Exec(t, `CREATE DATABASE d`) 190 191 sinkURL, cleanup := sqlutils.PGUrl(t, s.ServingSQLAddr(), t.Name(), url.User(security.RootUser)) 192 defer cleanup() 193 sinkURL.Path = `d` 194 195 targets := jobspb.ChangefeedTargets{ 196 0: jobspb.ChangefeedTarget{StatementTimeName: `foo`}, 197 1: jobspb.ChangefeedTarget{StatementTimeName: `bar`}, 198 } 199 sink, err := makeSQLSink(sinkURL.String(), `sink`, targets) 200 require.NoError(t, err) 201 defer func() { require.NoError(t, sink.Close()) }() 202 203 // Empty 204 require.NoError(t, sink.Flush(ctx)) 205 206 // Undeclared topic 207 require.EqualError(t, 208 sink.EmitRow(ctx, table(`nope`), nil, nil, zeroTS), `cannot emit to undeclared topic: nope`) 209 210 // With one row, nothing flushes until Flush is called. 211 require.NoError(t, sink.EmitRow(ctx, table(`foo`), []byte(`k1`), []byte(`v0`), zeroTS)) 212 sqlDB.CheckQueryResults(t, `SELECT key, value FROM sink ORDER BY PRIMARY KEY sink`, 213 [][]string{}, 214 ) 215 require.NoError(t, sink.Flush(ctx)) 216 sqlDB.CheckQueryResults(t, `SELECT key, value FROM sink ORDER BY PRIMARY KEY sink`, 217 [][]string{{`k1`, `v0`}}, 218 ) 219 sqlDB.Exec(t, `TRUNCATE sink`) 220 221 // Verify the implicit flushing 222 sqlDB.CheckQueryResults(t, `SELECT count(*) FROM sink`, [][]string{{`0`}}) 223 for i := 0; i < sqlSinkRowBatchSize+1; i++ { 224 require.NoError(t, 225 sink.EmitRow(ctx, table(`foo`), []byte(`k1`), []byte(`v`+strconv.Itoa(i)), zeroTS)) 226 } 227 // Should have auto flushed after sqlSinkRowBatchSize 228 sqlDB.CheckQueryResults(t, `SELECT count(*) FROM sink`, [][]string{{`3`}}) 229 require.NoError(t, sink.Flush(ctx)) 230 sqlDB.CheckQueryResults(t, `SELECT count(*) FROM sink`, [][]string{{`4`}}) 231 sqlDB.Exec(t, `TRUNCATE sink`) 232 233 // Two tables interleaved in time 234 require.NoError(t, sink.EmitRow(ctx, table(`foo`), []byte(`kfoo`), []byte(`v0`), zeroTS)) 235 require.NoError(t, sink.EmitRow(ctx, table(`bar`), []byte(`kbar`), []byte(`v0`), zeroTS)) 236 require.NoError(t, sink.EmitRow(ctx, table(`foo`), []byte(`kfoo`), []byte(`v1`), zeroTS)) 237 require.NoError(t, sink.Flush(ctx)) 238 sqlDB.CheckQueryResults(t, `SELECT topic, key, value FROM sink ORDER BY PRIMARY KEY sink`, 239 [][]string{{`bar`, `kbar`, `v0`}, {`foo`, `kfoo`, `v0`}, {`foo`, `kfoo`, `v1`}}, 240 ) 241 sqlDB.Exec(t, `TRUNCATE sink`) 242 243 // Multiple keys interleaved in time. Use sqlSinkNumPartitions+1 keys to 244 // guarantee that at lease two of them end up in the same partition. 245 for i := 0; i < sqlSinkNumPartitions+1; i++ { 246 require.NoError(t, 247 sink.EmitRow(ctx, table(`foo`), []byte(`v`+strconv.Itoa(i)), []byte(`v0`), zeroTS)) 248 } 249 for i := 0; i < sqlSinkNumPartitions+1; i++ { 250 require.NoError(t, 251 sink.EmitRow(ctx, table(`foo`), []byte(`v`+strconv.Itoa(i)), []byte(`v1`), zeroTS)) 252 } 253 require.NoError(t, sink.Flush(ctx)) 254 sqlDB.CheckQueryResults(t, `SELECT partition, key, value FROM sink ORDER BY PRIMARY KEY sink`, 255 [][]string{ 256 {`0`, `v3`, `v0`}, 257 {`0`, `v3`, `v1`}, 258 {`1`, `v1`, `v0`}, 259 {`1`, `v2`, `v0`}, 260 {`1`, `v1`, `v1`}, 261 {`1`, `v2`, `v1`}, 262 {`2`, `v0`, `v0`}, 263 {`2`, `v0`, `v1`}, 264 }, 265 ) 266 sqlDB.Exec(t, `TRUNCATE sink`) 267 268 // Emit resolved 269 var e testEncoder 270 require.NoError(t, sink.EmitResolvedTimestamp(ctx, e, zeroTS)) 271 require.NoError(t, sink.EmitRow(ctx, table(`foo`), []byte(`foo0`), []byte(`v0`), zeroTS)) 272 require.NoError(t, sink.EmitResolvedTimestamp(ctx, e, hlc.Timestamp{WallTime: 1})) 273 require.NoError(t, sink.Flush(ctx)) 274 sqlDB.CheckQueryResults(t, 275 `SELECT topic, partition, key, value, resolved FROM sink ORDER BY PRIMARY KEY sink`, 276 [][]string{ 277 {`bar`, `0`, ``, ``, `0,0`}, 278 {`bar`, `0`, ``, ``, `0.000000001,0`}, 279 {`bar`, `1`, ``, ``, `0,0`}, 280 {`bar`, `1`, ``, ``, `0.000000001,0`}, 281 {`bar`, `2`, ``, ``, `0,0`}, 282 {`bar`, `2`, ``, ``, `0.000000001,0`}, 283 {`foo`, `0`, ``, ``, `0,0`}, 284 {`foo`, `0`, `foo0`, `v0`, ``}, 285 {`foo`, `0`, ``, ``, `0.000000001,0`}, 286 {`foo`, `1`, ``, ``, `0,0`}, 287 {`foo`, `1`, ``, ``, `0.000000001,0`}, 288 {`foo`, `2`, ``, ``, `0,0`}, 289 {`foo`, `2`, ``, ``, `0.000000001,0`}, 290 }, 291 ) 292 }