| 12345678910111213141516171819202122232425262728293031323334353637383940414243444546474849505152535455565758596061626364656667686970717273747576777879808182838485868788899091929394959697989910010110210310410510610710810911011111211311411511611711811912012112212312412512612712812913013113213313413513613713813914014114214314414514614714814915015115215315415515615715815916016116216316416516616716816917017117217317417517617717817918018118218318418518618718818919019119219319419519619719819920020120220320420520620720820921021121221321421521621721821922022122222322422522622722822923023123223323423523623723823924024124224324424524624724824925025125225325425525625725825926026126226326426526626726826927027127227327427527627727827928028128228328428528628728828929029129229329429529629729829930030130230330430530630730830931031131231331431531631731831932032132232332432532632732832933033133233333433533633733833934034134234334434534634734834935035135235335435535635735835936036136236336436536636736836937037137237337437537637737837938038138238338438538638738838939039139239339439539639739839940040140240340440540640740840941041141241341441541641741841942042142242342442542642742842943043143243343443543643743843944044144244344444544644744844945045145245345445545645745845946046146246346446546646746846947047147247347447547647747847948048148248348448548648748848949049149249349449549649749849950050150250350450550650750850951051151251351451551651751851952052152252352452552652752852953053153253353453553653753853954054154254354454554654754854955055155255355455555655755855956056156256356456556656756856957057157257357457557657757857958058158258358458558658758858959059159259359459559659759859960060160260360460560660760860961061161261361461561661761861962062162262362462562662762862963063163263363463563663763863964064164264364464564664764864965065165265365465565665765865966066166266366466566666766866967067167267367467567667767867968068168268368468568668768868969069169269369469569669769869970070170270370470570670770870971071171271371471571671771871972072172272372472572672772872973073173273373473573673773873974074174274374474574674774874975075175275375475575675775875976076176276376476576676776876977077177277377477577677777877978078178278378478578678778878979079179279379479579679779879980080180280380480580680780880981081181281381481581681781881982082182282382482582682782882983083183283383483583683783883984084184284384484584684784884985085185285385485585685785885986086186286386486586686786886987087187287387487587687787887988088188288388488588688788888989089189289389489589689789889990090190290390490590690790890991091191291391491591691791891992092192292392492592692792892993093193293393493593693793893994094194294394494594694794894995095195295395495595695795895996096196296396496596696796896997097197297397497597697797897998098198298398498598698798898999099199299399499599699799899910001001100210031004 |
- // Copyright (c) 2012 The gocql Authors. All rights reserved.
- // Use of this source code is governed by a BSD-style
- // license that can be found in the LICENSE file.
- // +build all unit
- package gocql
- import (
- "bufio"
- "bytes"
- "context"
- "crypto/tls"
- "crypto/x509"
- "fmt"
- "io"
- "io/ioutil"
- "net"
- "os"
- "strings"
- "sync"
- "sync/atomic"
- "testing"
- "time"
- "github.com/gocql/gocql/internal/streams"
- )
- const (
- defaultProto = protoVersion2
- )
- func TestApprove(t *testing.T) {
- tests := map[bool]bool{
- approve("org.apache.cassandra.auth.PasswordAuthenticator"): true,
- approve("com.instaclustr.cassandra.auth.SharedSecretAuthenticator"): true,
- approve("com.datastax.bdp.cassandra.auth.DseAuthenticator"): true,
- approve("com.apache.cassandra.auth.FakeAuthenticator"): false,
- }
- for k, v := range tests {
- if k != v {
- t.Fatalf("expected '%v', got '%v'", k, v)
- }
- }
- }
- func TestJoinHostPort(t *testing.T) {
- tests := map[string]string{
- "127.0.0.1:0": JoinHostPort("127.0.0.1", 0),
- "127.0.0.1:1": JoinHostPort("127.0.0.1:1", 9142),
- "[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:0": JoinHostPort("2001:0db8:85a3:0000:0000:8a2e:0370:7334", 0),
- "[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:1": JoinHostPort("[2001:0db8:85a3:0000:0000:8a2e:0370:7334]:1", 9142),
- }
- for k, v := range tests {
- if k != v {
- t.Fatalf("expected '%v', got '%v'", k, v)
- }
- }
- }
- func testCluster(addr string, proto protoVersion) *ClusterConfig {
- cluster := NewCluster(addr)
- cluster.ProtoVersion = int(proto)
- cluster.disableControlConn = true
- return cluster
- }
- func TestSimple(t *testing.T) {
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, defaultProto)
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
- }
- if err := db.Query("void").Exec(); err != nil {
- t.Fatalf("0x%x: %v", defaultProto, err)
- }
- }
- func TestSSLSimple(t *testing.T) {
- srv := NewSSLTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- db, err := createTestSslCluster(srv.Address, defaultProto, true).CreateSession()
- if err != nil {
- t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
- }
- if err := db.Query("void").Exec(); err != nil {
- t.Fatalf("0x%x: %v", defaultProto, err)
- }
- }
- func TestSSLSimpleNoClientCert(t *testing.T) {
- srv := NewSSLTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- db, err := createTestSslCluster(srv.Address, defaultProto, false).CreateSession()
- if err != nil {
- t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
- }
- if err := db.Query("void").Exec(); err != nil {
- t.Fatalf("0x%x: %v", defaultProto, err)
- }
- }
- func createTestSslCluster(addr string, proto protoVersion, useClientCert bool) *ClusterConfig {
- cluster := testCluster(addr, proto)
- sslOpts := &SslOptions{
- CaPath: "testdata/pki/ca.crt",
- EnableHostVerification: false,
- }
- if useClientCert {
- sslOpts.CertPath = "testdata/pki/gocql.crt"
- sslOpts.KeyPath = "testdata/pki/gocql.key"
- }
- cluster.SslOpts = sslOpts
- return cluster
- }
- func TestClosed(t *testing.T) {
- t.Skip("Skipping the execution of TestClosed for now to try to concentrate on more important test failures on Travis")
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- session, err := newTestSession(srv.Address, defaultProto)
- if err != nil {
- t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
- }
- session.Close()
- if err := session.Query("void").Exec(); err != ErrSessionClosed {
- t.Fatalf("0x%x: expected %#v, got %#v", defaultProto, ErrSessionClosed, err)
- }
- }
- func newTestSession(addr string, proto protoVersion) (*Session, error) {
- return testCluster(addr, proto).CreateSession()
- }
- func TestDNSLookupConnected(t *testing.T) {
- log := &testLogger{}
- Logger = log
- defer func() {
- Logger = &defaultLogger{}
- }()
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := NewCluster("cassandra1.invalid", srv.Address, "cassandra2.invalid")
- cluster.ProtoVersion = int(defaultProto)
- cluster.disableControlConn = true
- // CreateSession() should attempt to resolve the DNS name "cassandraX.invalid"
- // and fail, but continue to connect via srv.Address
- _, err := cluster.CreateSession()
- if err != nil {
- t.Fatal("CreateSession() should have connected")
- }
- if !strings.Contains(log.String(), "gocql: dns error") {
- t.Fatalf("Expected to receive dns error log message - got '%s' instead", log.String())
- }
- }
- func TestDNSLookupError(t *testing.T) {
- log := &testLogger{}
- Logger = log
- defer func() {
- Logger = &defaultLogger{}
- }()
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := NewCluster("cassandra1.invalid", "cassandra2.invalid")
- cluster.ProtoVersion = int(defaultProto)
- cluster.disableControlConn = true
- // CreateSession() should attempt to resolve each DNS name "cassandraX.invalid"
- // and fail since it could not resolve any dns entries
- _, err := cluster.CreateSession()
- if err == nil {
- t.Fatal("CreateSession() should have returned an error")
- }
- if !strings.Contains(log.String(), "gocql: dns error") {
- t.Fatalf("Expected to receive dns error log message - got '%s' instead", log.String())
- }
- if err.Error() != "gocql: unable to create session: failed to resolve any of the provided hostnames" {
- t.Fatalf("Expected CreateSession() to fail with message - got '%s' instead", err.Error())
- }
- }
- func TestStartupTimeout(t *testing.T) {
- ctx, cancel := context.WithCancel(context.Background())
- log := &testLogger{}
- Logger = log
- defer func() {
- Logger = &defaultLogger{}
- }()
- srv := NewTestServer(t, defaultProto, ctx)
- defer srv.Stop()
- // Tell the server to never respond to Startup frame
- atomic.StoreInt32(&srv.TimeoutOnStartup, 1)
- startTime := time.Now()
- cluster := NewCluster(srv.Address)
- cluster.ProtoVersion = int(defaultProto)
- cluster.disableControlConn = true
- // Set very long query connection timeout
- // so we know CreateSession() is using the ConnectTimeout
- cluster.Timeout = time.Second * 5
- // Create session should timeout during connect attempt
- _, err := cluster.CreateSession()
- if err == nil {
- t.Fatal("CreateSession() should have returned a timeout error")
- }
- elapsed := time.Since(startTime)
- if elapsed > time.Second*5 {
- t.Fatal("ConnectTimeout is not respected")
- }
- if !strings.Contains(err.Error(), "no connections were made when creating the session") {
- t.Fatalf("Expected to receive no connections error - got '%s'", err)
- }
- if !strings.Contains(log.String(), "no response to connection startup within timeout") {
- t.Fatalf("Expected to receive timeout log message - got '%s'", log.String())
- }
- cancel()
- }
- func TestTimeout(t *testing.T) {
- ctx, cancel := context.WithCancel(context.Background())
- srv := NewTestServer(t, defaultProto, ctx)
- defer srv.Stop()
- db, err := newTestSession(srv.Address, defaultProto)
- if err != nil {
- t.Fatalf("NewCluster: %v", err)
- }
- defer db.Close()
- var wg sync.WaitGroup
- wg.Add(1)
- go func() {
- defer wg.Done()
- select {
- case <-time.After(5 * time.Second):
- t.Errorf("no timeout")
- case <-ctx.Done():
- }
- }()
- if err := db.Query("kill").WithContext(ctx).Exec(); err == nil {
- t.Fatal("expected error got nil")
- }
- cancel()
- wg.Wait()
- }
- type testRetryPolicy struct {
- numRetries int // maximum number of times to retry a query
- attemptTimeout time.Duration
- t *testing.T
- }
- // Attempt tells gocql to attempt the query again based on query.Attempts being less
- // than the NumRetries defined in the policy.
- func (s *testRetryPolicy) Attempt(q RetryableQuery) bool {
- return q.Attempts() <= s.numRetries
- }
- func (s *testRetryPolicy) GetRetryType(err error) RetryType {
- return Retry
- }
- // AttemptTimeout satisfies the optional RetryPolicyWithAttemptTimeout interface.
- func (s *testRetryPolicy) AttemptTimeout() time.Duration {
- return s.attemptTimeout
- }
- type testQueryObserver struct{}
- func (o *testQueryObserver) ObserveQuery(ctx context.Context, q ObservedQuery) {
- Logger.Printf("Observed query %q. Returned %v rows, took %v on host %q. Error: %q\n", q.Statement, q.Rows, q.End.Sub(q.Start), q.Host.ConnectAddress().String(), q.Err)
- }
- // TestQueryRetry will test to make sure that gocql will execute
- // the exact amount of retry queries designated by the user.
- func TestQueryRetry(t *testing.T) {
- ctx, cancel := context.WithCancel(context.Background())
- defer cancel()
- log := &testLogger{}
- Logger = log
- defer func() {
- Logger = &defaultLogger{}
- os.Stdout.WriteString(log.String())
- }()
- srv := NewTestServer(t, defaultProto, ctx)
- defer srv.Stop()
- db, err := newTestSession(srv.Address, defaultProto)
- if err != nil {
- t.Fatalf("NewCluster: %v", err)
- }
- defer db.Close()
- go func() {
- select {
- case <-ctx.Done():
- return
- case <-time.After(5 * time.Second):
- t.Errorf("no timeout")
- }
- }()
- rt := &testRetryPolicy{numRetries: 10, t: t, attemptTimeout: time.Millisecond * 25}
- queryCtx, cancel := context.WithTimeout(context.Background(), time.Millisecond*90)
- defer cancel()
- qry := db.Query("slow").RetryPolicy(rt).Observer(&testQueryObserver{}).WithContext(queryCtx)
- if err := qry.Exec(); err == nil {
- t.Fatalf("expected error")
- }
- // wait for the last slow query to finish
- // this prevents the test from flaking because of writing to a connection that's been closed
- time.Sleep(100 * time.Millisecond)
- numQueries := atomic.LoadUint64(&srv.nQueries)
- // the 90ms timeout allows at most 4 retries
- if numQueries > 4 {
- t.Fatalf("Too many retries executed for query. Query executed %v times", numQueries)
- }
- // make sure query is retried to guard against regressions
- if numQueries < 2 {
- t.Fatalf("Not enough retries executed for query. Query executed %v times", numQueries)
- }
- }
- func TestStreams_Protocol1(t *testing.T) {
- srv := NewTestServer(t, protoVersion1, context.Background())
- defer srv.Stop()
- // TODO: these are more like session tests and should instead operate
- // on a single Conn
- cluster := testCluster(srv.Address, protoVersion1)
- cluster.NumConns = 1
- cluster.ProtoVersion = 1
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatal(err)
- }
- defer db.Close()
- var wg sync.WaitGroup
- for i := 1; i < 128; i++ {
- // here were just validating that if we send NumStream request we get
- // a response for every stream and the lengths for the queries are set
- // correctly.
- wg.Add(1)
- go func() {
- defer wg.Done()
- if err := db.Query("void").Exec(); err != nil {
- t.Error(err)
- }
- }()
- }
- wg.Wait()
- }
- func TestStreams_Protocol3(t *testing.T) {
- srv := NewTestServer(t, protoVersion3, context.Background())
- defer srv.Stop()
- // TODO: these are more like session tests and should instead operate
- // on a single Conn
- cluster := testCluster(srv.Address, protoVersion3)
- cluster.NumConns = 1
- cluster.ProtoVersion = 3
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatal(err)
- }
- defer db.Close()
- for i := 1; i < 32768; i++ {
- // the test server processes each conn synchronously
- // here were just validating that if we send NumStream request we get
- // a response for every stream and the lengths for the queries are set
- // correctly.
- if err = db.Query("void").Exec(); err != nil {
- t.Fatal(err)
- }
- }
- }
- func BenchmarkProtocolV3(b *testing.B) {
- srv := NewTestServer(b, protoVersion3, context.Background())
- defer srv.Stop()
- // TODO: these are more like session tests and should instead operate
- // on a single Conn
- cluster := NewCluster(srv.Address)
- cluster.NumConns = 1
- cluster.ProtoVersion = 3
- db, err := cluster.CreateSession()
- if err != nil {
- b.Fatal(err)
- }
- defer db.Close()
- b.ResetTimer()
- b.ReportAllocs()
- for i := 0; i < b.N; i++ {
- if err = db.Query("void").Exec(); err != nil {
- b.Fatal(err)
- }
- }
- }
- // This tests that the policy connection pool handles SSL correctly
- func TestPolicyConnPoolSSL(t *testing.T) {
- srv := NewSSLTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := createTestSslCluster(srv.Address, defaultProto, true)
- cluster.PoolConfig.HostSelectionPolicy = RoundRobinHostPolicy()
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatalf("failed to create new session: %v", err)
- }
- if err := db.Query("void").Exec(); err != nil {
- t.Fatalf("query failed due to error: %v", err)
- }
- db.Close()
- // wait for the pool to drain
- time.Sleep(100 * time.Millisecond)
- size := db.pool.Size()
- if size != 0 {
- t.Fatalf("connection pool did not drain, still contains %d connections", size)
- }
- }
- func TestQueryTimeout(t *testing.T) {
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, defaultProto)
- // Set the timeout arbitrarily low so that the query hits the timeout in a
- // timely manner.
- cluster.Timeout = 1 * time.Millisecond
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatalf("NewCluster: %v", err)
- }
- defer db.Close()
- ch := make(chan error, 1)
- go func() {
- err := db.Query("timeout").Exec()
- if err != nil {
- ch <- err
- return
- }
- t.Errorf("err was nil, expected to get a timeout after %v", db.cfg.Timeout)
- }()
- select {
- case err := <-ch:
- if err != ErrTimeoutNoResponse {
- t.Fatalf("expected to get %v for timeout got %v", ErrTimeoutNoResponse, err)
- }
- case <-time.After(10*time.Millisecond + db.cfg.Timeout):
- // ensure that the query goroutines have been scheduled
- t.Fatalf("query did not timeout after %v", db.cfg.Timeout)
- }
- }
- func BenchmarkSingleConn(b *testing.B) {
- srv := NewTestServer(b, 3, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, 3)
- // Set the timeout arbitrarily low so that the query hits the timeout in a
- // timely manner.
- cluster.Timeout = 500 * time.Millisecond
- cluster.NumConns = 1
- db, err := cluster.CreateSession()
- if err != nil {
- b.Fatalf("NewCluster: %v", err)
- }
- defer db.Close()
- b.ResetTimer()
- b.RunParallel(func(pb *testing.PB) {
- for pb.Next() {
- err := db.Query("void").Exec()
- if err != nil {
- b.Error(err)
- return
- }
- }
- })
- }
- func TestQueryTimeoutReuseStream(t *testing.T) {
- t.Skip("no longer tests anything")
- // TODO(zariel): move this to conn test, we really just want to check what
- // happens when a conn is
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, defaultProto)
- // Set the timeout arbitrarily low so that the query hits the timeout in a
- // timely manner.
- cluster.Timeout = 1 * time.Millisecond
- cluster.NumConns = 1
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatalf("NewCluster: %v", err)
- }
- defer db.Close()
- db.Query("slow").Exec()
- err = db.Query("void").Exec()
- if err != nil {
- t.Fatal(err)
- }
- }
- func TestQueryTimeoutClose(t *testing.T) {
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, defaultProto)
- // Set the timeout arbitrarily low so that the query hits the timeout in a
- // timely manner.
- cluster.Timeout = 1000 * time.Millisecond
- cluster.NumConns = 1
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatalf("NewCluster: %v", err)
- }
- ch := make(chan error)
- go func() {
- err := db.Query("timeout").Exec()
- ch <- err
- }()
- // ensure that the above goroutine gets sheduled
- time.Sleep(50 * time.Millisecond)
- db.Close()
- select {
- case err = <-ch:
- case <-time.After(1 * time.Second):
- t.Fatal("timedout waiting to get a response once cluster is closed")
- }
- if err != ErrConnectionClosed {
- t.Fatalf("expected to get %v got %v", ErrConnectionClosed, err)
- }
- }
- func TestStream0(t *testing.T) {
- // TODO: replace this with type check
- const expErr = "gocql: received unexpected frame on stream 0"
- var buf bytes.Buffer
- f := newFramer(nil, &buf, nil, protoVersion4)
- f.writeHeader(0, opResult, 0)
- f.writeInt(resultKindVoid)
- f.wbuf[0] |= 0x80
- if err := f.finishWrite(); err != nil {
- t.Fatal(err)
- }
- conn := &Conn{
- r: bufio.NewReader(&buf),
- streams: streams.New(protoVersion4),
- }
- err := conn.recv()
- if err == nil {
- t.Fatal("expected to get an error on stream 0")
- } else if !strings.HasPrefix(err.Error(), expErr) {
- t.Fatalf("expected to get error prefix %q got %q", expErr, err.Error())
- }
- }
- func TestConnClosedBlocked(t *testing.T) {
- t.Skip("FLAKE: skipping test flake see https://github.com/gocql/gocql/issues/1088")
- // issue 664
- const proto = 3
- srv := NewTestServer(t, proto, context.Background())
- defer srv.Stop()
- errorHandler := connErrorHandlerFn(func(conn *Conn, err error, closed bool) {
- t.Log(err)
- })
- s, err := srv.session()
- if err != nil {
- t.Fatal(err)
- }
- defer s.Close()
- conn, err := s.connect(srv.host(), errorHandler)
- if err != nil {
- t.Fatal(err)
- }
- if err := conn.conn.Close(); err != nil {
- t.Fatal(err)
- }
- // This will block indefintaly if #664 is not fixed
- err = conn.executeQuery(&Query{stmt: "void"}).Close()
- if !strings.HasSuffix(err.Error(), "use of closed network connection") {
- t.Fatalf("expected to get use of closed networking connection error got: %v\n", err)
- }
- }
- func TestContext_Timeout(t *testing.T) {
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, defaultProto)
- cluster.Timeout = 5 * time.Second
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatal(err)
- }
- defer db.Close()
- ctx, cancel := context.WithCancel(context.Background())
- cancel()
- err = db.Query("timeout").WithContext(ctx).Exec()
- if err != context.Canceled {
- t.Fatalf("expected to get context cancel error: %v got %v", context.Canceled, err)
- }
- }
- type recordingFrameHeaderObserver struct {
- t *testing.T
- mu sync.Mutex
- frames []ObservedFrameHeader
- }
- func (r *recordingFrameHeaderObserver) ObserveFrameHeader(ctx context.Context, frm ObservedFrameHeader) {
- r.mu.Lock()
- r.frames = append(r.frames, frm)
- r.mu.Unlock()
- }
- func (r *recordingFrameHeaderObserver) getFrames() []ObservedFrameHeader {
- r.mu.Lock()
- defer r.mu.Unlock()
- return r.frames
- }
- func TestFrameHeaderObserver(t *testing.T) {
- srv := NewTestServer(t, defaultProto, context.Background())
- defer srv.Stop()
- cluster := testCluster(srv.Address, defaultProto)
- cluster.NumConns = 1
- observer := &recordingFrameHeaderObserver{t: t}
- cluster.FrameHeaderObserver = observer
- db, err := cluster.CreateSession()
- if err != nil {
- t.Fatal(err)
- }
- if err := db.Query("void").Exec(); err != nil {
- t.Fatal(err)
- }
- frames := observer.getFrames()
- if len(frames) != 2 {
- t.Fatalf("Expected to receive 2 frames, instead received %d", len(frames))
- }
- readyFrame := frames[0]
- if readyFrame.Opcode != frameOp(opReady) {
- t.Fatalf("Expected to receive ready frame, instead received frame of opcode %d", readyFrame.Opcode)
- }
- voidResultFrame := frames[1]
- if voidResultFrame.Opcode != frameOp(opResult) {
- t.Fatalf("Expected to receive result frame, instead received frame of opcode %d", voidResultFrame.Opcode)
- }
- if voidResultFrame.Length != int32(4) {
- t.Fatalf("Expected to receive frame with body length 4, instead received body length %d", voidResultFrame.Length)
- }
- }
- func NewTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
- laddr, err := net.ResolveTCPAddr("tcp", "127.0.0.1:0")
- if err != nil {
- t.Fatal(err)
- }
- listen, err := net.ListenTCP("tcp", laddr)
- if err != nil {
- t.Fatal(err)
- }
- headerSize := 8
- if protocol > protoVersion2 {
- headerSize = 9
- }
- ctx, cancel := context.WithCancel(ctx)
- srv := &TestServer{
- Address: listen.Addr().String(),
- listen: listen,
- t: t,
- protocol: protocol,
- headerSize: headerSize,
- ctx: ctx,
- cancel: cancel,
- }
- go srv.closeWatch()
- go srv.serve()
- return srv
- }
- func NewSSLTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
- pem, err := ioutil.ReadFile("testdata/pki/ca.crt")
- certPool := x509.NewCertPool()
- if !certPool.AppendCertsFromPEM(pem) {
- t.Fatalf("Failed parsing or appending certs")
- }
- mycert, err := tls.LoadX509KeyPair("testdata/pki/cassandra.crt", "testdata/pki/cassandra.key")
- if err != nil {
- t.Fatalf("could not load cert")
- }
- config := &tls.Config{
- Certificates: []tls.Certificate{mycert},
- RootCAs: certPool,
- }
- listen, err := tls.Listen("tcp", "127.0.0.1:0", config)
- if err != nil {
- t.Fatal(err)
- }
- headerSize := 8
- if protocol > protoVersion2 {
- headerSize = 9
- }
- ctx, cancel := context.WithCancel(ctx)
- srv := &TestServer{
- Address: listen.Addr().String(),
- listen: listen,
- t: t,
- protocol: protocol,
- headerSize: headerSize,
- ctx: ctx,
- cancel: cancel,
- }
- go srv.closeWatch()
- go srv.serve()
- return srv
- }
- type TestServer struct {
- Address string
- TimeoutOnStartup int32
- t testing.TB
- nreq uint64
- listen net.Listener
- nKillReq int64
- nQueries uint64
- compressor Compressor
- protocol byte
- headerSize int
- ctx context.Context
- cancel context.CancelFunc
- quit chan struct{}
- mu sync.Mutex
- closed bool
- }
- func (srv *TestServer) session() (*Session, error) {
- return testCluster(srv.Address, protoVersion(srv.protocol)).CreateSession()
- }
- func (srv *TestServer) host() *HostInfo {
- hosts, err := hostInfo(srv.Address, 9042)
- if err != nil {
- srv.t.Fatal(err)
- }
- return hosts[0]
- }
- func (srv *TestServer) closeWatch() {
- <-srv.ctx.Done()
- srv.mu.Lock()
- defer srv.mu.Unlock()
- srv.closeLocked()
- }
- func (srv *TestServer) serve() {
- defer srv.listen.Close()
- for !srv.isClosed() {
- conn, err := srv.listen.Accept()
- if err != nil {
- break
- }
- go func(conn net.Conn) {
- defer conn.Close()
- for !srv.isClosed() {
- framer, err := srv.readFrame(conn)
- if err != nil {
- if err == io.EOF {
- return
- }
- srv.errorLocked(err)
- return
- }
- atomic.AddUint64(&srv.nreq, 1)
- go srv.process(framer)
- }
- }(conn)
- }
- }
- func (srv *TestServer) isClosed() bool {
- srv.mu.Lock()
- defer srv.mu.Unlock()
- return srv.closed
- }
- func (srv *TestServer) closeLocked() {
- if srv.closed {
- return
- }
- srv.closed = true
- srv.listen.Close()
- srv.cancel()
- }
- func (srv *TestServer) Stop() {
- srv.mu.Lock()
- defer srv.mu.Unlock()
- srv.closeLocked()
- }
- func (srv *TestServer) errorLocked(err interface{}) {
- srv.mu.Lock()
- defer srv.mu.Unlock()
- if srv.closed {
- return
- }
- srv.t.Error(err)
- }
- func (srv *TestServer) process(f *framer) {
- head := f.header
- if head == nil {
- srv.errorLocked("process frame with a nil header")
- return
- }
- switch head.op {
- case opStartup:
- if atomic.LoadInt32(&srv.TimeoutOnStartup) > 0 {
- // Do not respond to startup command
- // wait until we get a cancel signal
- select {
- case <-srv.ctx.Done():
- return
- }
- }
- f.writeHeader(0, opReady, head.stream)
- case opOptions:
- f.writeHeader(0, opSupported, head.stream)
- f.writeShort(0)
- case opQuery:
- atomic.AddUint64(&srv.nQueries, 1)
- query := f.readLongString()
- first := query
- if n := strings.Index(query, " "); n > 0 {
- first = first[:n]
- }
- switch strings.ToLower(first) {
- case "kill":
- atomic.AddInt64(&srv.nKillReq, 1)
- f.writeHeader(0, opError, head.stream)
- f.writeInt(0x1001)
- f.writeString("query killed")
- case "use":
- f.writeInt(resultKindKeyspace)
- f.writeString(strings.TrimSpace(query[3:]))
- case "void":
- f.writeHeader(0, opResult, head.stream)
- f.writeInt(resultKindVoid)
- case "timeout":
- <-srv.ctx.Done()
- return
- case "slow":
- go func() {
- f.writeHeader(0, opResult, head.stream)
- f.writeInt(resultKindVoid)
- f.wbuf[0] = srv.protocol | 0x80
- select {
- case <-srv.ctx.Done():
- return
- case <-time.After(50 * time.Millisecond):
- f.finishWrite()
- }
- }()
- return
- default:
- f.writeHeader(0, opResult, head.stream)
- f.writeInt(resultKindVoid)
- }
- case opError:
- f.writeHeader(0, opError, head.stream)
- f.wbuf = append(f.wbuf, f.rbuf...)
- default:
- f.writeHeader(0, opError, head.stream)
- f.writeInt(0)
- f.writeString("not supported")
- }
- f.wbuf[0] = srv.protocol | 0x80
- if err := f.finishWrite(); err != nil {
- srv.errorLocked(err)
- }
- }
- func (srv *TestServer) readFrame(conn net.Conn) (*framer, error) {
- buf := make([]byte, srv.headerSize)
- head, err := readHeader(conn, buf)
- if err != nil {
- return nil, err
- }
- framer := newFramer(conn, conn, nil, srv.protocol)
- err = framer.readFrame(&head)
- if err != nil {
- return nil, err
- }
- // should be a request frame
- if head.version.response() {
- return nil, fmt.Errorf("expected to read a request frame got version: %v", head.version)
- } else if head.version.version() != srv.protocol {
- return nil, fmt.Errorf("expected to read protocol version 0x%x got 0x%x", srv.protocol, head.version.version())
- }
- return framer, nil
- }
|