Browse Source

Query metric per host 1155 (#1156)

* Store attempts and latencies in a host-based map

Signed-off-by: Alex Lourie <alex@instaclustr.com>

* Update ObservedQuery for the new metrics.

Signed-off-by: Alex Lourie <alex@instaclustr.com>

* Tests improvements

* Now it's possible to spin multi-node test clusters
* Batch tests moved from legacy to session.<> calls.

Signed-off-by: Alex Lourie <alex@instaclustr.com>

* Added test for query with metrics and QueryObserver

Signed-off-by: Alex Lourie <alex@instaclustr.com>

* Test on Travis with newer C* version and add self to contributors

Signed-off-by: Alex Lourie <alex@instaclustr.com>

* Initiating values in the same way

Signed-off-by: Alex Lourie <alex@instaclustr.com>
Alex Lourie 7 năm trước cách đây
mục cha
commit
4d29881651
8 tập tin đã thay đổi với 199 bổ sung57 xóa
  1. 2 0
      .travis.yml
  2. 1 0
      AUTHORS
  3. 5 5
      cassandra_test.go
  4. 101 20
      conn_test.go
  5. 2 1
      control.go
  6. 4 2
      policies_test.go
  7. 70 23
      session.go
  8. 14 6
      session_test.go

+ 2 - 0
.travis.yml

@@ -27,6 +27,8 @@ env:
       AUTH=false
     - CASS=3.11.3
       AUTH=false
+    - CASS=3.11.3
+      AUTH=false
 
 go:
   - "1.9"

+ 1 - 0
AUTHORS

@@ -106,3 +106,4 @@ Chang Liu <changliu.it@gmail.com>
 Ingo Oeser <nightlyone@gmail.com>
 Luke Hines <lukehines@protonmail.com>
 Jacob Greenleaf <jacob@jacobgreenleaf.com>
+Alex Lourie <alex@instaclustr.com>; <djay.il@gmail.com>

+ 5 - 5
cassandra_test.go

@@ -591,7 +591,7 @@ func TestBatch(t *testing.T) {
 		t.Fatal("create table:", err)
 	}
 
-	batch := NewBatch(LoggedBatch)
+	batch := session.NewBatch(LoggedBatch)
 	for i := 0; i < 100; i++ {
 		batch.Query(`INSERT INTO batch_table (id) VALUES (?)`, i)
 	}
@@ -623,9 +623,9 @@ func TestUnpreparedBatch(t *testing.T) {
 
 	var batch *Batch
 	if session.cfg.ProtoVersion == 2 {
-		batch = NewBatch(CounterBatch)
+		batch = session.NewBatch(CounterBatch)
 	} else {
-		batch = NewBatch(UnloggedBatch)
+		batch = session.NewBatch(UnloggedBatch)
 	}
 
 	for i := 0; i < 100; i++ {
@@ -664,7 +664,7 @@ func TestBatchLimit(t *testing.T) {
 		t.Fatal("create table:", err)
 	}
 
-	batch := NewBatch(LoggedBatch)
+	batch := session.NewBatch(LoggedBatch)
 	for i := 0; i < 65537; i++ {
 		batch.Query(`INSERT INTO batch_table2 (id) VALUES (?)`, i)
 	}
@@ -1869,7 +1869,7 @@ func TestBatchObserve(t *testing.T) {
 
 	var observedBatch *observation
 
-	batch := NewBatch(LoggedBatch)
+	batch := session.NewBatch(LoggedBatch)
 	batch.Observer(funcBatchObserver(func(ctx context.Context, o ObservedBatch) {
 		if observedBatch != nil {
 			t.Fatal("batch observe called more than once")

+ 101 - 20
conn_test.go

@@ -56,8 +56,8 @@ func TestJoinHostPort(t *testing.T) {
 	}
 }
 
-func testCluster(addr string, proto protoVersion) *ClusterConfig {
-	cluster := NewCluster(addr)
+func testCluster(proto protoVersion, addresses ...string) *ClusterConfig {
+	cluster := NewCluster(addresses...)
 	cluster.ProtoVersion = int(proto)
 	cluster.disableControlConn = true
 	return cluster
@@ -67,7 +67,7 @@ func TestSimple(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, defaultProto)
+	cluster := testCluster(defaultProto, srv.Address)
 	db, err := cluster.CreateSession()
 	if err != nil {
 		t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
@@ -107,7 +107,7 @@ func TestSSLSimpleNoClientCert(t *testing.T) {
 }
 
 func createTestSslCluster(addr string, proto protoVersion, useClientCert bool) *ClusterConfig {
-	cluster := testCluster(addr, proto)
+	cluster := testCluster(proto, addr)
 	sslOpts := &SslOptions{
 		CaPath:                 "testdata/pki/ca.crt",
 		EnableHostVerification: false,
@@ -128,7 +128,7 @@ func TestClosed(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	session, err := newTestSession(srv.Address, defaultProto)
+	session, err := newTestSession(defaultProto, srv.Address)
 	if err != nil {
 		t.Fatalf("0x%x: NewCluster: %v", defaultProto, err)
 	}
@@ -140,8 +140,8 @@ func TestClosed(t *testing.T) {
 	}
 }
 
-func newTestSession(addr string, proto protoVersion) (*Session, error) {
-	return testCluster(addr, proto).CreateSession()
+func newTestSession(proto protoVersion, addresses ...string) (*Session, error) {
+	return testCluster(proto, addresses...).CreateSession()
 }
 
 func TestDNSLookupConnected(t *testing.T) {
@@ -255,7 +255,7 @@ func TestTimeout(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, ctx)
 	defer srv.Stop()
 
-	db, err := newTestSession(srv.Address, defaultProto)
+	db, err := newTestSession(defaultProto, srv.Address)
 	if err != nil {
 		t.Fatalf("NewCluster: %v", err)
 	}
@@ -282,6 +282,24 @@ func TestTimeout(t *testing.T) {
 	wg.Wait()
 }
 
+type testQueryObserver struct {
+	metrics map[string]*queryMetrics
+	verbose bool
+}
+
+func (o *testQueryObserver) ObserveQuery(ctx context.Context, q ObservedQuery) {
+	host := q.Host.ConnectAddress().String()
+	o.metrics[host] = q.Metrics
+	if o.verbose {
+		Logger.Printf("Observed query %q. Returned %v rows, took %v on host %q with %v attempts and total latency %v. Error: %q\n",
+			q.Statement, q.Rows, q.End.Sub(q.Start), host, q.Metrics.Attempts, q.Metrics.TotalLatency, q.Err)
+	}
+}
+
+func (o *testQueryObserver) GetMetrics(host *HostInfo) *queryMetrics {
+	return o.metrics[host.ConnectAddress().String()]
+}
+
 // 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) {
@@ -291,7 +309,7 @@ func TestQueryRetry(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, ctx)
 	defer srv.Stop()
 
-	db, err := newTestSession(srv.Address, defaultProto)
+	db, err := newTestSession(defaultProto, srv.Address)
 	if err != nil {
 		t.Fatalf("NewCluster: %v", err)
 	}
@@ -325,13 +343,72 @@ func TestQueryRetry(t *testing.T) {
 	}
 }
 
+func TestQueryMultinodeWithMetrics(t *testing.T) {
+
+	// Build a 3 node cluster to test host metric mapping
+	var nodes []*TestServer
+	var addresses = []string{
+		"127.0.0.1",
+		"127.0.0.2",
+		"127.0.0.3",
+	}
+	// Can do with 1 context for all servers
+	ctx := context.Background()
+	for _, ip := range addresses {
+		srv := NewTestServerWithAddress(ip+":0", t, defaultProto, ctx)
+		defer srv.Stop()
+		nodes = append(nodes, srv)
+	}
+
+	db, err := newTestSession(defaultProto, nodes[0].Address, nodes[1].Address, nodes[2].Address)
+	if err != nil {
+		t.Fatalf("NewCluster: %v", err)
+	}
+	defer db.Close()
+
+	// 1 retry per host
+	rt := &SimpleRetryPolicy{NumRetries: 3}
+	observer := &testQueryObserver{metrics: make(map[string]*queryMetrics), verbose: false}
+	qry := db.Query("kill").RetryPolicy(rt).Observer(observer)
+	if err := qry.Exec(); err == nil {
+		t.Fatalf("expected error")
+	}
+
+	for i, ip := range addresses {
+		host := &HostInfo{connectAddress: net.ParseIP(ip)}
+		observedMetrics := observer.GetMetrics(host)
+
+		requests := int(atomic.LoadInt64(&nodes[i].nKillReq))
+		hostAttempts := qry.metrics[ip].Attempts
+		if requests != hostAttempts {
+			t.Fatalf("expected requests %v to match query attempts %v", requests, hostAttempts)
+		}
+
+		if hostAttempts != observedMetrics.Attempts {
+			t.Fatalf("expected observed attempts %v to match query attempts %v on host %v", observedMetrics.Attempts, hostAttempts, ip)
+		}
+
+		hostLatency := qry.metrics[ip].TotalLatency
+		observedLatency := observedMetrics.TotalLatency
+		if hostLatency != observedLatency {
+			t.Fatalf("expected observed latency %v to match query latency %v on host %v", observedLatency, hostLatency, ip)
+		}
+	}
+	// the query will only be attempted once, but is being retried
+	attempts := qry.Attempts()
+	if attempts != rt.NumRetries {
+		t.Fatalf("failed to retry the query %v time(s). Query executed %v times", rt.NumRetries, attempts)
+	}
+
+}
+
 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 := testCluster(protoVersion1, srv.Address)
 	cluster.NumConns = 1
 	cluster.ProtoVersion = 1
 
@@ -363,7 +440,7 @@ func TestStreams_Protocol3(t *testing.T) {
 
 	// TODO: these are more like session tests and should instead operate
 	// on a single Conn
-	cluster := testCluster(srv.Address, protoVersion3)
+	cluster := testCluster(protoVersion3, srv.Address)
 	cluster.NumConns = 1
 	cluster.ProtoVersion = 3
 
@@ -439,7 +516,7 @@ func TestQueryTimeout(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, defaultProto)
+	cluster := testCluster(defaultProto, srv.Address)
 	// Set the timeout arbitrarily low so that the query hits the timeout in a
 	// timely manner.
 	cluster.Timeout = 1 * time.Millisecond
@@ -476,7 +553,7 @@ func BenchmarkSingleConn(b *testing.B) {
 	srv := NewTestServer(b, 3, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, 3)
+	cluster := testCluster(3, srv.Address)
 	// Set the timeout arbitrarily low so that the query hits the timeout in a
 	// timely manner.
 	cluster.Timeout = 500 * time.Millisecond
@@ -507,7 +584,7 @@ func TestQueryTimeoutReuseStream(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, defaultProto)
+	cluster := testCluster(defaultProto, srv.Address)
 	// Set the timeout arbitrarily low so that the query hits the timeout in a
 	// timely manner.
 	cluster.Timeout = 1 * time.Millisecond
@@ -531,7 +608,7 @@ func TestQueryTimeoutClose(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, defaultProto)
+	cluster := testCluster(defaultProto, srv.Address)
 	// Set the timeout arbitrarily low so that the query hits the timeout in a
 	// timely manner.
 	cluster.Timeout = 1000 * time.Millisecond
@@ -625,7 +702,7 @@ func TestContext_Timeout(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, defaultProto)
+	cluster := testCluster(defaultProto, srv.Address)
 	cluster.Timeout = 5 * time.Second
 	db, err := cluster.CreateSession()
 	if err != nil {
@@ -663,7 +740,7 @@ func TestFrameHeaderObserver(t *testing.T) {
 	srv := NewTestServer(t, defaultProto, context.Background())
 	defer srv.Stop()
 
-	cluster := testCluster(srv.Address, defaultProto)
+	cluster := testCluster(defaultProto, srv.Address)
 	cluster.NumConns = 1
 	observer := &recordingFrameHeaderObserver{t: t}
 	cluster.FrameHeaderObserver = observer
@@ -695,8 +772,8 @@ func TestFrameHeaderObserver(t *testing.T) {
 	}
 }
 
-func NewTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
-	laddr, err := net.ResolveTCPAddr("tcp", "127.0.0.1:0")
+func NewTestServerWithAddress(addr string, t testing.TB, protocol uint8, ctx context.Context) *TestServer {
+	laddr, err := net.ResolveTCPAddr("tcp", addr)
 	if err != nil {
 		t.Fatal(err)
 	}
@@ -728,6 +805,10 @@ func NewTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServe
 	return srv
 }
 
+func NewTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
+	return NewTestServerWithAddress("127.0.0.1:0", t, protocol, ctx)
+}
+
 func NewSSLTestServer(t testing.TB, protocol uint8, ctx context.Context) *TestServer {
 	pem, err := ioutil.ReadFile("testdata/pki/ca.crt")
 	certPool := x509.NewCertPool()
@@ -788,7 +869,7 @@ type TestServer struct {
 }
 
 func (srv *TestServer) session() (*Session, error) {
-	return testCluster(srv.Address, protoVersion(srv.protocol)).CreateSession()
+	return testCluster(protoVersion(srv.protocol), srv.Address).CreateSession()
 }
 
 func (srv *TestServer) host() *HostInfo {

+ 2 - 1
control.go

@@ -453,7 +453,8 @@ func (c *controlConn) query(statement string, values ...interface{}) (iter *Iter
 			Logger.Printf("control: error executing %q: %v\n", statement, iter.err)
 		}
 
-		q.attempts++
+		metric := q.getHostMetrics(c.getConn().host)
+		metric.Attempts++
 		if iter.err == nil || !c.retry.Attempt(q) {
 			break
 		}

+ 4 - 2
policies_test.go

@@ -263,8 +263,9 @@ func TestSimpleRetryPolicy(t *testing.T) {
 		{5, false},
 	}
 
+	q.metrics = make(map[string]*queryMetrics)
 	for _, c := range cases {
-		q.attempts = c.attempts
+		q.metrics["127.0.0.1"] = &queryMetrics{Attempts: c.attempts}
 		if c.allow && !rt.Attempt(q) {
 			t.Fatalf("should allow retry after %d attempts", c.attempts)
 		}
@@ -347,8 +348,9 @@ func TestDowngradingConsistencyRetryPolicy(t *testing.T) {
 		{16, false, reu1, Retry},
 	}
 
+	q.metrics = make(map[string]*queryMetrics)
 	for _, c := range cases {
-		q.attempts = c.attempts
+		q.metrics["127.0.0.1"] = &queryMetrics{Attempts: c.attempts}
 		if c.retryType != rt.GetRetryType(c.err) {
 			t.Fatalf("retry type should be %v", c.retryType)
 		}

+ 70 - 23
session.go

@@ -658,6 +658,11 @@ func (s *Session) connect(host *HostInfo, errorHandler ConnErrorHandler) (*Conn,
 	return s.dial(host, s.connCfg, errorHandler)
 }
 
+type queryMetrics struct {
+	Attempts     int
+	TotalLatency int64
+}
+
 // Query represents a CQL statement that can be executed.
 type Query struct {
 	stmt                  string
@@ -673,14 +678,13 @@ type Query struct {
 	session               *Session
 	rt                    RetryPolicy
 	binding               func(q *QueryInfo) ([]interface{}, error)
-	attempts              int
-	totalLatency          int64
 	serialCons            SerialConsistency
 	defaultTimestamp      bool
 	defaultTimestampValue int64
 	disableSkipMetadata   bool
 	context               context.Context
 	idempotent            bool
+	metrics               map[string]*queryMetrics
 
 	disableAutoPage bool
 }
@@ -698,9 +702,21 @@ func (q *Query) defaultsFromSession() {
 	q.serialCons = s.cfg.SerialConsistency
 	q.defaultTimestamp = s.cfg.DefaultTimestamp
 	q.idempotent = s.cfg.DefaultIdempotence
+	q.metrics = make(map[string]*queryMetrics)
 	s.mu.RUnlock()
 }
 
+func (q *Query) getHostMetrics(host *HostInfo) *queryMetrics {
+	hostMetrics, exists := q.metrics[host.ConnectAddress().String()]
+	if !exists {
+		// if the host is not in the map, it means it's been accessed for the first time
+		hostMetrics = &queryMetrics{Attempts: 0, TotalLatency: 0}
+		q.metrics[host.ConnectAddress().String()] = hostMetrics
+	}
+
+	return hostMetrics
+}
+
 // Statement returns the statement that was used to generate this query.
 func (q Query) Statement() string {
 	return q.stmt
@@ -713,13 +729,23 @@ func (q Query) String() string {
 
 //Attempts returns the number of times the query was executed.
 func (q *Query) Attempts() int {
-	return q.attempts
+	attempts := 0
+	for _, metric := range q.metrics {
+		attempts += metric.Attempts
+	}
+	return attempts
 }
 
 //Latency returns the average amount of nanoseconds per attempt of the query.
 func (q *Query) Latency() int64 {
-	if q.attempts > 0 {
-		return q.totalLatency / int64(q.attempts)
+	var attempts int
+	var latency int64
+	for _, metric := range q.metrics {
+		attempts += metric.Attempts
+		latency += metric.TotalLatency
+	}
+	if attempts > 0 {
+		return latency / int64(attempts)
 	}
 	return 0
 }
@@ -808,9 +834,9 @@ func (q *Query) execute(conn *Conn) *Iter {
 }
 
 func (q *Query) attempt(keyspace string, end, start time.Time, iter *Iter, host *HostInfo) {
-	q.attempts++
-	q.totalLatency += end.Sub(start).Nanoseconds()
-	// TODO: track latencies per host and things as well instead of just total
+	hostMetrics := q.getHostMetrics(host)
+	hostMetrics.Attempts++
+	hostMetrics.TotalLatency += end.Sub(start).Nanoseconds()
 
 	if q.observer != nil {
 		q.observer.ObserveQuery(q.context, ObservedQuery{
@@ -820,8 +846,8 @@ func (q *Query) attempt(keyspace string, end, start time.Time, iter *Iter, host
 			End:       end,
 			Rows:      iter.numRows,
 			Host:      host,
+			Metrics:   hostMetrics,
 			Err:       iter.err,
-			Attempt:   q.attempts,
 		})
 	}
 }
@@ -1388,13 +1414,12 @@ type Batch struct {
 	Cons                  Consistency
 	rt                    RetryPolicy
 	observer              BatchObserver
-	attempts              int
-	totalLatency          int64
 	serialCons            SerialConsistency
 	defaultTimestamp      bool
 	defaultTimestampValue int64
 	context               context.Context
 	keyspace              string
+	metrics               map[string]*queryMetrics
 }
 
 // NewBatch creates a new batch operation without defaults from the cluster
@@ -1415,11 +1440,23 @@ func (s *Session) NewBatch(typ BatchType) *Batch {
 		Cons:             s.cons,
 		defaultTimestamp: s.cfg.DefaultTimestamp,
 		keyspace:         s.cfg.Keyspace,
+		metrics:          make(map[string]*queryMetrics),
 	}
 	s.mu.RUnlock()
 	return batch
 }
 
+func (b *Batch) getHostMetrics(host *HostInfo) *queryMetrics {
+	hostMetrics, exists := b.metrics[host.ConnectAddress().String()]
+	if !exists {
+		// if the host is not in the map, it means it's been accessed for the first time
+		hostMetrics = &queryMetrics{Attempts: 0, TotalLatency: 0}
+		b.metrics[host.ConnectAddress().String()] = hostMetrics
+	}
+
+	return hostMetrics
+}
+
 // Observer enables batch-level observer on this batch.
 // The provided observer will be called every time this batched query is executed.
 func (b *Batch) Observer(observer BatchObserver) *Batch {
@@ -1433,13 +1470,23 @@ func (b *Batch) Keyspace() string {
 
 // Attempts returns the number of attempts made to execute the batch.
 func (b *Batch) Attempts() int {
-	return b.attempts
+	attempts := 0
+	for _, metric := range b.metrics {
+		attempts += metric.Attempts
+	}
+	return attempts
 }
 
 //Latency returns the average number of nanoseconds to execute a single attempt of the batch.
 func (b *Batch) Latency() int64 {
-	if b.attempts > 0 {
-		return b.totalLatency / int64(b.attempts)
+	attempts := 0
+	var latency int64 = 0
+	for _, metric := range b.metrics {
+		attempts += metric.Attempts
+		latency += metric.TotalLatency
+	}
+	if attempts > 0 {
+		return latency / int64(attempts)
 	}
 	return 0
 }
@@ -1526,9 +1573,9 @@ func (b *Batch) WithTimestamp(timestamp int64) *Batch {
 }
 
 func (b *Batch) attempt(keyspace string, end, start time.Time, iter *Iter, host *HostInfo) {
-	b.attempts++
-	b.totalLatency += end.Sub(start).Nanoseconds()
-	// TODO: track latencies per host and things as well instead of just total
+	hostMetrics := b.getHostMetrics(host)
+	hostMetrics.Attempts++
+	hostMetrics.TotalLatency += end.Sub(start).Nanoseconds()
 
 	if b.observer == nil {
 		return
@@ -1546,8 +1593,8 @@ func (b *Batch) attempt(keyspace string, end, start time.Time, iter *Iter, host
 		End:        end,
 		// Rows not used in batch observations // TODO - might be able to support it when using BatchCAS
 		Host:    host,
+		Metrics: hostMetrics,
 		Err:     iter.err,
-		Attempt: b.attempts,
 	})
 }
 
@@ -1699,12 +1746,12 @@ type ObservedQuery struct {
 	// Host is the informations about the host that performed the query
 	Host *HostInfo
 
+	// The metrics per this host
+	Metrics *queryMetrics
+
 	// Err is the error in the query.
 	// It only tracks network errors or errors of bad cassandra syntax, in particular selects with no match return nil error
 	Err error
-
-	// Attempt contains the number of times the query has been attempted so far.
-	Attempt int
 }
 
 // QueryObserver is the interface implemented by query observers / stat collectors.
@@ -1731,8 +1778,8 @@ type ObservedBatch struct {
 	// It only tracks network errors or errors of bad cassandra syntax, in particular selects with no match return nil error
 	Err error
 
-	// Attempt contains the number of times the query has been attempted so far.
-	Attempt int
+	// The metrics per this host
+	Metrics *queryMetrics
 }
 
 // BatchObserver is the interface implemented by batch observers / stat collectors.

+ 14 - 6
session_test.go

@@ -99,12 +99,18 @@ func (f funcQueryObserver) ObserveQuery(ctx context.Context, o ObservedQuery) {
 func TestQueryBasicAPI(t *testing.T) {
 	qry := &Query{}
 
+	// Initialise metrics map
+	qry.metrics = make(map[string]*queryMetrics)
+
+	// Initiate host
+	ip := "127.0.0.1"
+
+	qry.metrics[ip] = &queryMetrics{Attempts: 0, TotalLatency: 0}
 	if qry.Latency() != 0 {
 		t.Fatalf("expected Query.Latency() to return 0, got %v", qry.Latency())
 	}
 
-	qry.attempts = 2
-	qry.totalLatency = 4
+	qry.metrics[ip] = &queryMetrics{Attempts: 2, TotalLatency: 4}
 	if qry.Attempts() != 2 {
 		t.Fatalf("expected Query.Attempts() to return 2, got %v", qry.Attempts())
 	}
@@ -191,15 +197,17 @@ func TestBatchBasicAPI(t *testing.T) {
 	}
 
 	// Test LoggedBatch
-	b = NewBatch(LoggedBatch)
+	b = s.NewBatch(LoggedBatch)
 	if b.Type != LoggedBatch {
 		t.Fatalf("expected batch.Type to be '%v', got '%v'", LoggedBatch, b.Type)
 	}
 
+	ip := "127.0.0.1"
+
 	// Test attempts
-	b.attempts = 1
+	b.metrics[ip] = &queryMetrics{Attempts: 1}
 	if b.Attempts() != 1 {
-		t.Fatalf("expceted batch.Attempts() to return %v, got %v", 1, b.Attempts())
+		t.Fatalf("expected batch.Attempts() to return %v, got %v", 1, b.Attempts())
 	}
 
 	// Test latency
@@ -207,7 +215,7 @@ func TestBatchBasicAPI(t *testing.T) {
 		t.Fatalf("expected batch.Latency() to be 0, got %v", b.Latency())
 	}
 
-	b.totalLatency = 4
+	b.metrics[ip] = &queryMetrics{Attempts: 1, TotalLatency: 4}
 	if b.Latency() != 4 {
 		t.Fatalf("expected batch.Latency() to return %v, got %v", 4, b.Latency())
 	}