conn_test.go 23 KB

123456789101112131415161718192021222324252627282930313233343536373839404142434445464748495051525354555657585960616263646566676869707172737475767778798081828384858687888990919293949596979899100101102103104105106107108109110111112113114115116117118119120121122123124125126127128129130131132133134135136137138139140141142143144145146147148149150151152153154155156157158159160161162163164165166167168169170171172173174175176177178179180181182183184185186187188189190191192193194195196197198199200201202203204205206207208209210211212213214215216217218219220221222223224225226227228229230231232233234235236237238239240241242243244245246247248249250251252253254255256257258259260261262263264265266267268269270271272273274275276277278279280281282283284285286287288289290291292293294295296297298299300301302303304305306307308309310311312313314315316317318319320321322323324325326327328329330331332333334335336337338339340341342343344345346347348349350351352353354355356357358359360361362363364365366367368369370371372373374375376377378379380381382383384385386387388389390391392393394395396397398399400401402403404405406407408409410411412413414415416417418419420421422423424425426427428429430431432433434435436437438439440441442443444445446447448449450451452453454455456457458459460461462463464465466467468469470471472473474475476477478479480481482483484485486487488489490491492493494495496497498499500501502503504505506507508509510511512513514515516517518519520521522523524525526527528529530531532533534535536537538539540541542543544545546547548549550551552553554555556557558559560561562563564565566567568569570571572573574575576577578579580581582583584585586587588589590591592593594595596597598599600601602603604605606607608609610611612613614615616617618619620621622623624625626627628629630631632633634635636637638639640641642643644645646647648649650651652653654655656657658659660661662663664665666667668669670671672673674675676677678679680681682683684685686687688689690691692693694695696697698699700701702703704705706707708709710711712713714715716717718719720721722723724725726727728729730731732733734735736737738739740741742743744745746747748749750751752753754755756757758759760761762763764765766767768769770771772773774775776777778779780781782783784785786787788789790791792793794795796797798799800801802803804805806807808809810811812813814815816817818819820821822823824825826827828829830831832833834835836837838839840841842843844845846847848849850851852853854855856857858859860861862863864865866867868869870871872873874875876877878879880881882883884885886887888889890891892893894895896897898899900901902903904905906907908909910911912913914915916917918919920921922923924925926927928929930931932933934935936937938939940941942943944945946947948949
  1. // Copyright (c) 2012 The gocql Authors. All rights reserved.
  2. // Use of this source code is governed by a BSD-style
  3. // license that can be found in the LICENSE file.
  4. // +build all unit
  5. package gocql
  6. import (
  7. "bufio"
  8. "bytes"
  9. "context"
  10. "crypto/tls"
  11. "crypto/x509"
  12. "fmt"
  13. "io"
  14. "io/ioutil"
  15. "net"
  16. "strings"
  17. "sync"
  18. "sync/atomic"
  19. "testing"
  20. "time"
  21. "github.com/gocql/gocql/internal/streams"
  22. )
  23. const (
  24. defaultProto = protoVersion2
  25. )
  26. func TestApprove(t *testing.T) {
  27. tests := map[bool]bool{
  28. approve("org.apache.cassandra.auth.PasswordAuthenticator"): true,
  29. approve("com.instaclustr.cassandra.auth.SharedSecretAuthenticator"): true,
  30. approve("com.datastax.bdp.cassandra.auth.DseAuthenticator"): true,
  31. approve("com.apache.cassandra.auth.FakeAuthenticator"): false,
  32. }
  33. for k, v := range tests {
  34. if k != v {
  35. t.Fatalf("expected '%v', got '%v'", k, v)
  36. }
  37. }
  38. }
  39. func TestJoinHostPort(t *testing.T) {
  40. tests := map[string]string{
  41. "127.0.0.1:0": JoinHostPort("127.0.0.1", 0),
  42. "127.0.0.1:1": JoinHostPort("127.0.0.1:1", 9142),
  43. "[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:0": JoinHostPort("2001:0db8:85a3:0000:0000:8a2e:0370:7334", 0),
  44. "[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:1": JoinHostPort("[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:1", 9142),
  45. }
  46. for k, v := range tests {
  47. if k != v {
  48. t.Fatalf("expected '%v', got '%v'", k, v)
  49. }
  50. }
  51. }
  52. func testCluster(addr string, proto protoVersion) *ClusterConfig {
  53. cluster := NewCluster(addr)
  54. cluster.ProtoVersion = int(proto)
  55. cluster.disableControlConn = true
  56. return cluster
  57. }
  58. func TestSimple(t *testing.T) {
  59. srv := NewTestServer(t, defaultProto, context.Background())
  60. defer srv.Stop()
  61. cluster := testCluster(srv.Address, defaultProto)
  62. db, err := cluster.CreateSession()
  63. if err != nil {
  64. t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
  65. }
  66. if err := db.Query("void").Exec(); err != nil {
  67. t.Fatalf("0x%x: %v", defaultProto, err)
  68. }
  69. }
  70. func TestSSLSimple(t *testing.T) {
  71. srv := NewSSLTestServer(t, defaultProto, context.Background())
  72. defer srv.Stop()
  73. db, err := createTestSslCluster(srv.Address, defaultProto, true).CreateSession()
  74. if err != nil {
  75. t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
  76. }
  77. if err := db.Query("void").Exec(); err != nil {
  78. t.Fatalf("0x%x: %v", defaultProto, err)
  79. }
  80. }
  81. func TestSSLSimpleNoClientCert(t *testing.T) {
  82. srv := NewSSLTestServer(t, defaultProto, context.Background())
  83. defer srv.Stop()
  84. db, err := createTestSslCluster(srv.Address, defaultProto, false).CreateSession()
  85. if err != nil {
  86. t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
  87. }
  88. if err := db.Query("void").Exec(); err != nil {
  89. t.Fatalf("0x%x: %v", defaultProto, err)
  90. }
  91. }
  92. func createTestSslCluster(addr string, proto protoVersion, useClientCert bool) *ClusterConfig {
  93. cluster := testCluster(addr, proto)
  94. sslOpts := &SslOptions{
  95. CaPath: "testdata/pki/ca.crt",
  96. EnableHostVerification: false,
  97. }
  98. if useClientCert {
  99. sslOpts.CertPath = "testdata/pki/gocql.crt"
  100. sslOpts.KeyPath = "testdata/pki/gocql.key"
  101. }
  102. cluster.SslOpts = sslOpts
  103. return cluster
  104. }
  105. func TestClosed(t *testing.T) {
  106. t.Skip("Skipping the execution of TestClosed for now to try to concentrate on more important test failures on Travis")
  107. srv := NewTestServer(t, defaultProto, context.Background())
  108. defer srv.Stop()
  109. session, err := newTestSession(srv.Address, defaultProto)
  110. if err != nil {
  111. t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
  112. }
  113. session.Close()
  114. if err := session.Query("void").Exec(); err != ErrSessionClosed {
  115. t.Fatalf("0x%x: expected %#v, got %#v", defaultProto, ErrSessionClosed, err)
  116. }
  117. }
  118. func newTestSession(addr string, proto protoVersion) (*Session, error) {
  119. return testCluster(addr, proto).CreateSession()
  120. }
  121. func TestDNSLookupConnected(t *testing.T) {
  122. log := &testLogger{}
  123. Logger = log
  124. defer func() {
  125. Logger = &defaultLogger{}
  126. }()
  127. srv := NewTestServer(t, defaultProto, context.Background())
  128. defer srv.Stop()
  129. cluster := NewCluster("cassandra1.invalid", srv.Address, "cassandra2.invalid")
  130. cluster.ProtoVersion = int(defaultProto)
  131. cluster.disableControlConn = true
  132. // CreateSession() should attempt to resolve the DNS name "cassandraX.invalid"
  133. // and fail, but continue to connect via srv.Address
  134. _, err := cluster.CreateSession()
  135. if err != nil {
  136. t.Fatal("CreateSession() should have connected")
  137. }
  138. if !strings.Contains(log.String(), "gocql: dns error") {
  139. t.Fatalf("Expected to receive dns error log message - got '%s' instead", log.String())
  140. }
  141. }
  142. func TestDNSLookupError(t *testing.T) {
  143. log := &testLogger{}
  144. Logger = log
  145. defer func() {
  146. Logger = &defaultLogger{}
  147. }()
  148. srv := NewTestServer(t, defaultProto, context.Background())
  149. defer srv.Stop()
  150. cluster := NewCluster("cassandra1.invalid", "cassandra2.invalid")
  151. cluster.ProtoVersion = int(defaultProto)
  152. cluster.disableControlConn = true
  153. // CreateSession() should attempt to resolve each DNS name "cassandraX.invalid"
  154. // and fail since it could not resolve any dns entries
  155. _, err := cluster.CreateSession()
  156. if err == nil {
  157. t.Fatal("CreateSession() should have returned an error")
  158. }
  159. if !strings.Contains(log.String(), "gocql: dns error") {
  160. t.Fatalf("Expected to receive dns error log message - got '%s' instead", log.String())
  161. }
  162. if err.Error() != "gocql: unable to create session: failed to resolve any of the provided hostnames" {
  163. t.Fatalf("Expected CreateSession() to fail with message - got '%s' instead", err.Error())
  164. }
  165. }
  166. func TestStartupTimeout(t *testing.T) {
  167. ctx, cancel := context.WithCancel(context.Background())
  168. log := &testLogger{}
  169. Logger = log
  170. defer func() {
  171. Logger = &defaultLogger{}
  172. }()
  173. srv := NewTestServer(t, defaultProto, ctx)
  174. defer srv.Stop()
  175. // Tell the server to never respond to Startup frame
  176. atomic.StoreInt32(&srv.TimeoutOnStartup, 1)
  177. startTime := time.Now()
  178. cluster := NewCluster(srv.Address)
  179. cluster.ProtoVersion = int(defaultProto)
  180. cluster.disableControlConn = true
  181. // Set very long query connection timeout
  182. // so we know CreateSession() is using the ConnectTimeout
  183. cluster.Timeout = time.Second * 5
  184. // Create session should timeout during connect attempt
  185. _, err := cluster.CreateSession()
  186. if err == nil {
  187. t.Fatal("CreateSession() should have returned a timeout error")
  188. }
  189. elapsed := time.Since(startTime)
  190. if elapsed > time.Second*5 {
  191. t.Fatal("ConnectTimeout is not respected")
  192. }
  193. if !strings.Contains(err.Error(), "no connections were made when creating the session") {
  194. t.Fatalf("Expected to receive no connections error - got '%s'", err)
  195. }
  196. if !strings.Contains(log.String(), "no response to connection startup within timeout") {
  197. t.Fatalf("Expected to receive timeout log message - got '%s'", log.String())
  198. }
  199. cancel()
  200. }
  201. func TestTimeout(t *testing.T) {
  202. ctx, cancel := context.WithCancel(context.Background())
  203. srv := NewTestServer(t, defaultProto, ctx)
  204. defer srv.Stop()
  205. db, err := newTestSession(srv.Address, defaultProto)
  206. if err != nil {
  207. t.Fatalf("NewCluster: %v", err)
  208. }
  209. defer db.Close()
  210. var wg sync.WaitGroup
  211. wg.Add(1)
  212. go func() {
  213. defer wg.Done()
  214. select {
  215. case <-time.After(5 * time.Second):
  216. t.Errorf("no timeout")
  217. case <-ctx.Done():
  218. }
  219. }()
  220. if err := db.Query("kill").WithContext(ctx).Exec(); err == nil {
  221. t.Fatal("expected error got nil")
  222. }
  223. cancel()
  224. wg.Wait()
  225. }
  226. // TestQueryRetry will test to make sure that gocql will execute
  227. // the exact amount of retry queries designated by the user.
  228. func TestQueryRetry(t *testing.T) {
  229. ctx, cancel := context.WithCancel(context.Background())
  230. defer cancel()
  231. srv := NewTestServer(t, defaultProto, ctx)
  232. defer srv.Stop()
  233. db, err := newTestSession(srv.Address, defaultProto)
  234. if err != nil {
  235. t.Fatalf("NewCluster: %v", err)
  236. }
  237. defer db.Close()
  238. go func() {
  239. select {
  240. case <-ctx.Done():
  241. return
  242. case <-time.After(5 * time.Second):
  243. t.Errorf("no timeout")
  244. }
  245. }()
  246. rt := &SimpleRetryPolicy{NumRetries: 1}
  247. qry := db.Query("kill").RetryPolicy(rt)
  248. if err := qry.Exec(); err == nil {
  249. t.Fatalf("expected error")
  250. }
  251. requests := atomic.LoadInt64(&srv.nKillReq)
  252. attempts := qry.Attempts()
  253. if requests != int64(attempts) {
  254. t.Fatalf("expected requests %v to match query attempts %v", requests, attempts)
  255. }
  256. // the query will only be attempted once, but is being retried
  257. if requests != int64(rt.NumRetries) {
  258. t.Fatalf("failed to retry the query %v time(s). Query executed %v times", rt.NumRetries, requests-1)
  259. }
  260. }
  261. func TestStreams_Protocol1(t *testing.T) {
  262. srv := NewTestServer(t, protoVersion1, context.Background())
  263. defer srv.Stop()
  264. // TODO: these are more like session tests and should instead operate
  265. // on a single Conn
  266. cluster := testCluster(srv.Address, protoVersion1)
  267. cluster.NumConns = 1
  268. cluster.ProtoVersion = 1
  269. db, err := cluster.CreateSession()
  270. if err != nil {
  271. t.Fatal(err)
  272. }
  273. defer db.Close()
  274. var wg sync.WaitGroup
  275. for i := 1; i < 128; i++ {
  276. // here were just validating that if we send NumStream request we get
  277. // a response for every stream and the lengths for the queries are set
  278. // correctly.
  279. wg.Add(1)
  280. go func() {
  281. defer wg.Done()
  282. if err := db.Query("void").Exec(); err != nil {
  283. t.Error(err)
  284. }
  285. }()
  286. }
  287. wg.Wait()
  288. }
  289. func TestStreams_Protocol3(t *testing.T) {
  290. srv := NewTestServer(t, protoVersion3, context.Background())
  291. defer srv.Stop()
  292. // TODO: these are more like session tests and should instead operate
  293. // on a single Conn
  294. cluster := testCluster(srv.Address, protoVersion3)
  295. cluster.NumConns = 1
  296. cluster.ProtoVersion = 3
  297. db, err := cluster.CreateSession()
  298. if err != nil {
  299. t.Fatal(err)
  300. }
  301. defer db.Close()
  302. for i := 1; i < 32768; i++ {
  303. // the test server processes each conn synchronously
  304. // here were just validating that if we send NumStream request we get
  305. // a response for every stream and the lengths for the queries are set
  306. // correctly.
  307. if err = db.Query("void").Exec(); err != nil {
  308. t.Fatal(err)
  309. }
  310. }
  311. }
  312. func BenchmarkProtocolV3(b *testing.B) {
  313. srv := NewTestServer(b, protoVersion3, context.Background())
  314. defer srv.Stop()
  315. // TODO: these are more like session tests and should instead operate
  316. // on a single Conn
  317. cluster := NewCluster(srv.Address)
  318. cluster.NumConns = 1
  319. cluster.ProtoVersion = 3
  320. db, err := cluster.CreateSession()
  321. if err != nil {
  322. b.Fatal(err)
  323. }
  324. defer db.Close()
  325. b.ResetTimer()
  326. b.ReportAllocs()
  327. for i := 0; i < b.N; i++ {
  328. if err = db.Query("void").Exec(); err != nil {
  329. b.Fatal(err)
  330. }
  331. }
  332. }
  333. // This tests that the policy connection pool handles SSL correctly
  334. func TestPolicyConnPoolSSL(t *testing.T) {
  335. srv := NewSSLTestServer(t, defaultProto, context.Background())
  336. defer srv.Stop()
  337. cluster := createTestSslCluster(srv.Address, defaultProto, true)
  338. cluster.PoolConfig.HostSelectionPolicy = RoundRobinHostPolicy()
  339. db, err := cluster.CreateSession()
  340. if err != nil {
  341. t.Fatalf("failed to create new session: %v", err)
  342. }
  343. if err := db.Query("void").Exec(); err != nil {
  344. t.Fatalf("query failed due to error: %v", err)
  345. }
  346. db.Close()
  347. // wait for the pool to drain
  348. time.Sleep(100 * time.Millisecond)
  349. size := db.pool.Size()
  350. if size != 0 {
  351. t.Fatalf("connection pool did not drain, still contains %d connections", size)
  352. }
  353. }
  354. func TestQueryTimeout(t *testing.T) {
  355. srv := NewTestServer(t, defaultProto, context.Background())
  356. defer srv.Stop()
  357. cluster := testCluster(srv.Address, defaultProto)
  358. // Set the timeout arbitrarily low so that the query hits the timeout in a
  359. // timely manner.
  360. cluster.Timeout = 1 * time.Millisecond
  361. db, err := cluster.CreateSession()
  362. if err != nil {
  363. t.Fatalf("NewCluster: %v", err)
  364. }
  365. defer db.Close()
  366. ch := make(chan error, 1)
  367. go func() {
  368. err := db.Query("timeout").Exec()
  369. if err != nil {
  370. ch <- err
  371. return
  372. }
  373. t.Errorf("err was nil, expected to get a timeout after %v", db.cfg.Timeout)
  374. }()
  375. select {
  376. case err := <-ch:
  377. if err != ErrTimeoutNoResponse {
  378. t.Fatalf("expected to get %v for timeout got %v", ErrTimeoutNoResponse, err)
  379. }
  380. case <-time.After(10*time.Millisecond + db.cfg.Timeout):
  381. // ensure that the query goroutines have been scheduled
  382. t.Fatalf("query did not timeout after %v", db.cfg.Timeout)
  383. }
  384. }
  385. func BenchmarkSingleConn(b *testing.B) {
  386. srv := NewTestServer(b, 3, context.Background())
  387. defer srv.Stop()
  388. cluster := testCluster(srv.Address, 3)
  389. // Set the timeout arbitrarily low so that the query hits the timeout in a
  390. // timely manner.
  391. cluster.Timeout = 500 * time.Millisecond
  392. cluster.NumConns = 1
  393. db, err := cluster.CreateSession()
  394. if err != nil {
  395. b.Fatalf("NewCluster: %v", err)
  396. }
  397. defer db.Close()
  398. b.ResetTimer()
  399. b.RunParallel(func(pb *testing.PB) {
  400. for pb.Next() {
  401. err := db.Query("void").Exec()
  402. if err != nil {
  403. b.Error(err)
  404. return
  405. }
  406. }
  407. })
  408. }
  409. func TestQueryTimeoutReuseStream(t *testing.T) {
  410. t.Skip("no longer tests anything")
  411. // TODO(zariel): move this to conn test, we really just want to check what
  412. // happens when a conn is
  413. srv := NewTestServer(t, defaultProto, context.Background())
  414. defer srv.Stop()
  415. cluster := testCluster(srv.Address, defaultProto)
  416. // Set the timeout arbitrarily low so that the query hits the timeout in a
  417. // timely manner.
  418. cluster.Timeout = 1 * time.Millisecond
  419. cluster.NumConns = 1
  420. db, err := cluster.CreateSession()
  421. if err != nil {
  422. t.Fatalf("NewCluster: %v", err)
  423. }
  424. defer db.Close()
  425. db.Query("slow").Exec()
  426. err = db.Query("void").Exec()
  427. if err != nil {
  428. t.Fatal(err)
  429. }
  430. }
  431. func TestQueryTimeoutClose(t *testing.T) {
  432. srv := NewTestServer(t, defaultProto, context.Background())
  433. defer srv.Stop()
  434. cluster := testCluster(srv.Address, defaultProto)
  435. // Set the timeout arbitrarily low so that the query hits the timeout in a
  436. // timely manner.
  437. cluster.Timeout = 1000 * time.Millisecond
  438. cluster.NumConns = 1
  439. db, err := cluster.CreateSession()
  440. if err != nil {
  441. t.Fatalf("NewCluster: %v", err)
  442. }
  443. ch := make(chan error)
  444. go func() {
  445. err := db.Query("timeout").Exec()
  446. ch <- err
  447. }()
  448. // ensure that the above goroutine gets sheduled
  449. time.Sleep(50 * time.Millisecond)
  450. db.Close()
  451. select {
  452. case err = <-ch:
  453. case <-time.After(1 * time.Second):
  454. t.Fatal("timedout waiting to get a response once cluster is closed")
  455. }
  456. if err != ErrConnectionClosed {
  457. t.Fatalf("expected to get %v got %v", ErrConnectionClosed, err)
  458. }
  459. }
  460. func TestStream0(t *testing.T) {
  461. // TODO: replace this with type check
  462. const expErr = "gocql: received unexpected frame on stream 0"
  463. var buf bytes.Buffer
  464. f := newFramer(nil, &buf, nil, protoVersion4)
  465. f.writeHeader(0, opResult, 0)
  466. f.writeInt(resultKindVoid)
  467. f.wbuf[0] |= 0x80
  468. if err := f.finishWrite(); err != nil {
  469. t.Fatal(err)
  470. }
  471. conn := &Conn{
  472. r: bufio.NewReader(&buf),
  473. streams: streams.New(protoVersion4),
  474. }
  475. err := conn.recv()
  476. if err == nil {
  477. t.Fatal("expected to get an error on stream 0")
  478. } else if !strings.HasPrefix(err.Error(), expErr) {
  479. t.Fatalf("expected to get error prefix %q got %q", expErr, err.Error())
  480. }
  481. }
  482. func TestConnClosedBlocked(t *testing.T) {
  483. t.Skip("FLAKE: skipping test flake see https://github.com/gocql/gocql/issues/1088")
  484. // issue 664
  485. const proto = 3
  486. srv := NewTestServer(t, proto, context.Background())
  487. defer srv.Stop()
  488. errorHandler := connErrorHandlerFn(func(conn *Conn, err error, closed bool) {
  489. t.Log(err)
  490. })
  491. s, err := srv.session()
  492. if err != nil {
  493. t.Fatal(err)
  494. }
  495. defer s.Close()
  496. conn, err := s.connect(srv.host(), errorHandler)
  497. if err != nil {
  498. t.Fatal(err)
  499. }
  500. if err := conn.conn.Close(); err != nil {
  501. t.Fatal(err)
  502. }
  503. // This will block indefintaly if #664 is not fixed
  504. err = conn.executeQuery(&Query{stmt: "void"}).Close()
  505. if !strings.HasSuffix(err.Error(), "use of closed network connection") {
  506. t.Fatalf("expected to get use of closed networking connection error got: %v\n", err)
  507. }
  508. }
  509. func TestContext_Timeout(t *testing.T) {
  510. srv := NewTestServer(t, defaultProto, context.Background())
  511. defer srv.Stop()
  512. cluster := testCluster(srv.Address, defaultProto)
  513. cluster.Timeout = 5 * time.Second
  514. db, err := cluster.CreateSession()
  515. if err != nil {
  516. t.Fatal(err)
  517. }
  518. defer db.Close()
  519. ctx, cancel := context.WithCancel(context.Background())
  520. cancel()
  521. err = db.Query("timeout").WithContext(ctx).Exec()
  522. if err != context.Canceled {
  523. t.Fatalf("expected to get context cancel error: %v got %v", context.Canceled, err)
  524. }
  525. }
  526. type recordingFrameHeaderObserver struct {
  527. t *testing.T
  528. frames []ObservedFrameHeader
  529. }
  530. func (r *recordingFrameHeaderObserver) ObserveFrameHeader(ctx context.Context, frm ObservedFrameHeader) {
  531. r.frames = append(r.frames, frm)
  532. }
  533. func TestFrameHeaderObserver(t *testing.T) {
  534. srv := NewTestServer(t, defaultProto, context.Background())
  535. defer srv.Stop()
  536. cluster := testCluster(srv.Address, defaultProto)
  537. observer := &recordingFrameHeaderObserver{t: t}
  538. cluster.FrameHeaderObserver = observer
  539. db, err := cluster.CreateSession()
  540. if err != nil {
  541. t.Fatal(err)
  542. }
  543. if err := db.Query("void").Exec(); err != nil {
  544. t.Fatal(err)
  545. }
  546. if len(observer.frames) != 2 {
  547. t.Fatalf("Expected to receive 2 frames, instead received %d", len(observer.frames))
  548. }
  549. readyFrame := observer.frames[0]
  550. if readyFrame.Opcode != byte(opReady) {
  551. t.Fatalf("Expected to receive ready frame, instead received frame of opcode %d", readyFrame.Opcode)
  552. }
  553. voidResultFrame := observer.frames[1]
  554. if voidResultFrame.Opcode != byte(opResult) {
  555. t.Fatalf("Expected to receive result frame, instead received frame of opcode %d", voidResultFrame.Opcode)
  556. }
  557. if voidResultFrame.Length != int32(4) {
  558. t.Fatalf("Expected to receive frame with body length 4, instead received body length %d", voidResultFrame.Length)
  559. }
  560. }
  561. func NewTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
  562. laddr, err := net.ResolveTCPAddr("tcp", "127.0.0.1:0")
  563. if err != nil {
  564. t.Fatal(err)
  565. }
  566. listen, err := net.ListenTCP("tcp", laddr)
  567. if err != nil {
  568. t.Fatal(err)
  569. }
  570. headerSize := 8
  571. if protocol > protoVersion2 {
  572. headerSize = 9
  573. }
  574. ctx, cancel := context.WithCancel(ctx)
  575. srv := &TestServer{
  576. Address: listen.Addr().String(),
  577. listen: listen,
  578. t: t,
  579. protocol: protocol,
  580. headerSize: headerSize,
  581. ctx: ctx,
  582. cancel: cancel,
  583. }
  584. go srv.closeWatch()
  585. go srv.serve()
  586. return srv
  587. }
  588. func NewSSLTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
  589. pem, err := ioutil.ReadFile("testdata/pki/ca.crt")
  590. certPool := x509.NewCertPool()
  591. if !certPool.AppendCertsFromPEM(pem) {
  592. t.Fatalf("Failed parsing or appending certs")
  593. }
  594. mycert, err := tls.LoadX509KeyPair("testdata/pki/cassandra.crt", "testdata/pki/cassandra.key")
  595. if err != nil {
  596. t.Fatalf("could not load cert")
  597. }
  598. config := &tls.Config{
  599. Certificates: []tls.Certificate{mycert},
  600. RootCAs: certPool,
  601. }
  602. listen, err := tls.Listen("tcp", "127.0.0.1:0", config)
  603. if err != nil {
  604. t.Fatal(err)
  605. }
  606. headerSize := 8
  607. if protocol > protoVersion2 {
  608. headerSize = 9
  609. }
  610. ctx, cancel := context.WithCancel(ctx)
  611. srv := &TestServer{
  612. Address: listen.Addr().String(),
  613. listen: listen,
  614. t: t,
  615. protocol: protocol,
  616. headerSize: headerSize,
  617. ctx: ctx,
  618. cancel: cancel,
  619. }
  620. go srv.closeWatch()
  621. go srv.serve()
  622. return srv
  623. }
  624. type TestServer struct {
  625. Address string
  626. TimeoutOnStartup int32
  627. t testing.TB
  628. nreq uint64
  629. listen net.Listener
  630. nKillReq int64
  631. compressor Compressor
  632. protocol byte
  633. headerSize int
  634. ctx context.Context
  635. cancel context.CancelFunc
  636. quit chan struct{}
  637. mu sync.Mutex
  638. closed bool
  639. }
  640. func (srv *TestServer) session() (*Session, error) {
  641. return testCluster(srv.Address, protoVersion(srv.protocol)).CreateSession()
  642. }
  643. func (srv *TestServer) host() *HostInfo {
  644. hosts, err := hostInfo(srv.Address, 9042)
  645. if err != nil {
  646. srv.t.Fatal(err)
  647. }
  648. return hosts[0]
  649. }
  650. func (srv *TestServer) closeWatch() {
  651. <-srv.ctx.Done()
  652. srv.mu.Lock()
  653. defer srv.mu.Unlock()
  654. srv.closeLocked()
  655. }
  656. func (srv *TestServer) serve() {
  657. defer srv.listen.Close()
  658. for !srv.isClosed() {
  659. conn, err := srv.listen.Accept()
  660. if err != nil {
  661. break
  662. }
  663. go func(conn net.Conn) {
  664. defer conn.Close()
  665. for !srv.isClosed() {
  666. framer, err := srv.readFrame(conn)
  667. if err != nil {
  668. if err == io.EOF {
  669. return
  670. }
  671. srv.errorLocked(err)
  672. return
  673. }
  674. atomic.AddUint64(&srv.nreq, 1)
  675. go srv.process(framer)
  676. }
  677. }(conn)
  678. }
  679. }
  680. func (srv *TestServer) isClosed() bool {
  681. srv.mu.Lock()
  682. defer srv.mu.Unlock()
  683. return srv.closed
  684. }
  685. func (srv *TestServer) closeLocked() {
  686. if srv.closed {
  687. return
  688. }
  689. srv.closed = true
  690. srv.listen.Close()
  691. srv.cancel()
  692. }
  693. func (srv *TestServer) Stop() {
  694. srv.mu.Lock()
  695. defer srv.mu.Unlock()
  696. srv.closeLocked()
  697. }
  698. func (srv *TestServer) errorLocked(err interface{}) {
  699. srv.mu.Lock()
  700. defer srv.mu.Unlock()
  701. if srv.closed {
  702. return
  703. }
  704. srv.t.Error(err)
  705. }
  706. func (srv *TestServer) process(f *framer) {
  707. head := f.header
  708. if head == nil {
  709. srv.errorLocked("process frame with a nil header")
  710. return
  711. }
  712. switch head.op {
  713. case opStartup:
  714. if atomic.LoadInt32(&srv.TimeoutOnStartup) > 0 {
  715. // Do not respond to startup command
  716. // wait until we get a cancel signal
  717. select {
  718. case <-srv.ctx.Done():
  719. return
  720. }
  721. }
  722. f.writeHeader(0, opReady, head.stream)
  723. case opOptions:
  724. f.writeHeader(0, opSupported, head.stream)
  725. f.writeShort(0)
  726. case opQuery:
  727. query := f.readLongString()
  728. first := query
  729. if n := strings.Index(query, " "); n > 0 {
  730. first = first[:n]
  731. }
  732. switch strings.ToLower(first) {
  733. case "kill":
  734. atomic.AddInt64(&srv.nKillReq, 1)
  735. f.writeHeader(0, opError, head.stream)
  736. f.writeInt(0x1001)
  737. f.writeString("query killed")
  738. case "use":
  739. f.writeInt(resultKindKeyspace)
  740. f.writeString(strings.TrimSpace(query[3:]))
  741. case "void":
  742. f.writeHeader(0, opResult, head.stream)
  743. f.writeInt(resultKindVoid)
  744. case "timeout":
  745. <-srv.ctx.Done()
  746. return
  747. case "slow":
  748. go func() {
  749. f.writeHeader(0, opResult, head.stream)
  750. f.writeInt(resultKindVoid)
  751. f.wbuf[0] = srv.protocol | 0x80
  752. select {
  753. case <-srv.ctx.Done():
  754. return
  755. case <-time.After(50 * time.Millisecond):
  756. f.finishWrite()
  757. }
  758. }()
  759. return
  760. default:
  761. f.writeHeader(0, opResult, head.stream)
  762. f.writeInt(resultKindVoid)
  763. }
  764. case opError:
  765. f.writeHeader(0, opError, head.stream)
  766. f.wbuf = append(f.wbuf, f.rbuf...)
  767. default:
  768. f.writeHeader(0, opError, head.stream)
  769. f.writeInt(0)
  770. f.writeString("not supported")
  771. }
  772. f.wbuf[0] = srv.protocol | 0x80
  773. if err := f.finishWrite(); err != nil {
  774. srv.errorLocked(err)
  775. }
  776. }
  777. func (srv *TestServer) readFrame(conn net.Conn) (*framer, error) {
  778. buf := make([]byte, srv.headerSize)
  779. head, err := readHeader(conn, buf)
  780. if err != nil {
  781. return nil, err
  782. }
  783. framer := newFramer(conn, conn, nil, srv.protocol)
  784. err = framer.readFrame(&head)
  785. if err != nil {
  786. return nil, err
  787. }
  788. // should be a request frame
  789. if head.version.response() {
  790. return nil, fmt.Errorf("expected to read a request frame got version: %v", head.version)
  791. } else if head.version.version() != srv.protocol {
  792. return nil, fmt.Errorf("expected to read protocol version 0x%x got 0x%x", srv.protocol, head.version.version())
  793. }
  794. return framer, nil
  795. }