Browse Source

Improved unit test coverage and function documentation.

- Added document lines for test functions
- Improved unit test coverage for token.go, metadata.go
- Added unit tests for policyConnPool in connectionpool.go, more work is needed here in the future
- Fixed a typo in connectionpool.go
Justin Corpron 10 years ago
parent
commit
ef8e794e62
9 changed files with 691 additions and 97 deletions
  1. 8 0
      cassandra_test.go
  2. 99 3
      conn_test.go
  3. 10 1
      connectionpool.go
  4. 20 8
      metadata.go
  5. 159 34
      metadata_test.go
  6. 5 1
      policies.go
  7. 46 9
      policies_test.go
  8. 29 11
      token.go
  9. 315 30
      token_test.go

+ 8 - 0
cassandra_test.go

@@ -1508,6 +1508,7 @@ func TestEmptyTimestamp(t *testing.T) {
 	}
 }
 
+// Integration test of just querying for data from the system.schema_keyspace table
 func TestGetKeyspaceMetadata(t *testing.T) {
 	session := createSession(t)
 	defer session.Close()
@@ -1541,6 +1542,7 @@ func TestGetKeyspaceMetadata(t *testing.T) {
 	}
 }
 
+// Integration test of just querying for data from the system.schema_columnfamilies table
 func TestGetTableMetadata(t *testing.T) {
 	session := createSession(t)
 	defer session.Close()
@@ -1622,6 +1624,7 @@ func TestGetTableMetadata(t *testing.T) {
 	}
 }
 
+// Integration test of just querying for data from the system.schema_columns table
 func TestGetColumnMetadata(t *testing.T) {
 	session := createSession(t)
 	defer session.Close()
@@ -1723,6 +1726,7 @@ func TestGetColumnMetadata(t *testing.T) {
 	}
 }
 
+// Integration test of querying and composition the keyspace metadata
 func TestKeyspaceMetadata(t *testing.T) {
 	session := createSession(t)
 	defer session.Close()
@@ -1785,6 +1789,7 @@ func TestKeyspaceMetadata(t *testing.T) {
 	}
 }
 
+// Integration test of the routing key calculation
 func TestRoutingKey(t *testing.T) {
 	session := createSession(t)
 	defer session.Close()
@@ -1903,6 +1908,7 @@ func TestRoutingKey(t *testing.T) {
 	}
 }
 
+// Integration test of the token-aware policy-based connection pool
 func TestTokenAwareConnPool(t *testing.T) {
 	cluster := createCluster()
 	cluster.ConnPoolType = NewTokenAwareConnPool
@@ -1946,4 +1952,6 @@ func TestTokenAwareConnPool(t *testing.T) {
 	if err := iter.Close(); err != nil {
 		t.Errorf("iter failed with err: %v", err)
 	}
+
+	// TODO add verification that the query went to the correct host
 }

+ 99 - 3
conn_test.go

@@ -1,3 +1,6 @@
+// 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
@@ -177,7 +180,7 @@ func TestSlowQuery(t *testing.T) {
 	}
 }
 
-func TestRoundRobin(t *testing.T) {
+func TestSimplePoolRoundRobin(t *testing.T) {
 	servers := make([]*TestServer, 5)
 	addrs := make([]string, len(servers))
 	for n := 0; n < len(servers); n++ {
@@ -223,7 +226,7 @@ func TestRoundRobin(t *testing.T) {
 	}
 
 	if diff > 0 {
-		t.Fatal("diff:", diff)
+		t.Errorf("Expected 0 difference in usage but was %d", diff)
 	}
 }
 
@@ -258,7 +261,7 @@ func TestConnClosing(t *testing.T) {
 	conns := pool.Size()
 
 	if conns != numConns {
-		t.Fatalf("Expected to have %d connections but have %d", numConns, conns)
+		t.Errorf("Expected to have %d connections but have %d", numConns, conns)
 	}
 }
 
@@ -373,6 +376,99 @@ func BenchmarkProtocolV3(b *testing.B) {
 	}
 }
 
+func TestRoundRobinConnPoolRoundRobin(t *testing.T) {
+	// create 5 test servers
+	servers := make([]*TestServer, 5)
+	addrs := make([]string, len(servers))
+	for n := 0; n < len(servers); n++ {
+		servers[n] = NewTestServer(t, defaultProto)
+		addrs[n] = servers[n].Address
+		defer servers[n].Stop()
+	}
+
+	// create a new cluster using the policy-based round robin conn pool
+	cluster := NewCluster(addrs...)
+	cluster.ConnPoolType = NewRoundRobinConnPool
+
+	db, err := cluster.CreateSession()
+	if err != nil {
+		t.Fatalf("failed to create a new session: %v", err)
+	}
+
+	// Sleep to allow the pool to fill
+	time.Sleep(100 * time.Millisecond)
+
+	// run concurrent queries against the pool, server usage should
+	// be even
+	var wg sync.WaitGroup
+	wg.Add(5)
+	for n := 0; n < 5; n++ {
+		go func() {
+			for j := 0; j < 5; j++ {
+				if err := db.Query("void").Exec(); err != nil {
+					t.Errorf("Query failed with error: %v", err)
+				}
+			}
+			wg.Done()
+		}()
+	}
+	wg.Wait()
+
+	db.Close()
+
+	// wait for the pool to drain
+	time.Sleep(100 * time.Millisecond)
+	size := db.Pool.Size()
+	if size != 0 {
+		t.Errorf("connection pool did not drain, still contains %d connections", size)
+	}
+
+	// verify that server usage is even
+	diff := 0
+	for n := 1; n < len(servers); n++ {
+		d := 0
+		if servers[n].nreq > servers[n-1].nreq {
+			d = int(servers[n].nreq - servers[n-1].nreq)
+		} else {
+			d = int(servers[n-1].nreq - servers[n].nreq)
+		}
+		if d > diff {
+			diff = d
+		}
+	}
+
+	if diff > 0 {
+		t.Errorf("expected 0 difference in usage but was %d", diff)
+	}
+}
+
+// This tests that the policy connection pool handles SSL correctly
+func TestPolicyConnPoolSSL(t *testing.T) {
+	srv := NewSSLTestServer(t, defaultProto)
+	defer srv.Stop()
+
+	cluster := createTestSslCluster(srv.Address, defaultProto)
+	cluster.ConnPoolType = NewRoundRobinConnPool
+
+	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.Errorf("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.Errorf("connection pool did not drain, still contains %d connections", size)
+	}
+}
+
 func NewTestServer(t testing.TB, protocol uint8) *TestServer {
 	laddr, err := net.ResolveTCPAddr("tcp", "127.0.0.1:0")
 	if err != nil {

+ 10 - 1
connectionpool.go

@@ -1,3 +1,7 @@
+// 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.
+
 package gocql
 
 import (
@@ -776,7 +780,7 @@ func (pool *hostConnPool) fill() {
 }
 
 func (pool *hostConnPool) logConnectErr(err error) {
-	if opErr, ok := err.(*net.OpError); ok && (opErr.Op == "dial" || opErr.Op == "red") {
+	if opErr, ok := err.(*net.OpError); ok && (opErr.Op == "dial" || opErr.Op == "read") {
 		// connection refused
 		// these are typical during a node outage so avoid log spam.
 	} else if err != nil {
@@ -837,6 +841,11 @@ func (pool *hostConnPool) HandleError(conn *Conn, err error, closed bool) {
 	pool.mu.Lock()
 	defer pool.mu.Unlock()
 
+	if pool.closed {
+		// pool closed
+		return
+	}
+
 	// find the connection index
 	for i, candidate := range pool.conns {
 		if candidate == conn {

+ 20 - 8
metadata.go

@@ -87,6 +87,8 @@ type schemaDescriber struct {
 	cache map[string]*KeyspaceMetadata
 }
 
+// creates a session bound schema describer which will query and cache
+// keyspace metadata
 func newSchemaDescriber(session *Session) *schemaDescriber {
 	return &schemaDescriber{
 		session: session,
@@ -94,6 +96,8 @@ func newSchemaDescriber(session *Session) *schemaDescriber {
 	}
 }
 
+// returns the cached KeyspaceMetadata held by the describer for the named
+// keyspace.
 func (s *schemaDescriber) getSchema(keyspaceName string) (*KeyspaceMetadata, error) {
 	s.mu.Lock()
 	defer s.mu.Unlock()
@@ -114,6 +118,8 @@ func (s *schemaDescriber) getSchema(keyspaceName string) (*KeyspaceMetadata, err
 	return metadata, nil
 }
 
+// forcibly updates the current KeyspaceMetadata held by the schema describer
+// for a given named keyspace.
 func (s *schemaDescriber) refreshSchema(keyspaceName string) error {
 	var err error
 
@@ -141,9 +147,11 @@ func (s *schemaDescriber) refreshSchema(keyspaceName string) error {
 	return nil
 }
 
-// "compiles" keyspace, table, and column metadata for a keyspace together
-// linking the metadata objects together and calculating the partition key
-// and clustering key.
+// "compiles" derived information about keyspace, table, and column metadata
+// for a keyspace from the basic queried metadata objects returned by
+// getKeyspaceMetadata, getTableMetadata, and getColumnMetadata respectively;
+// Links the metadata objects together and derives the column composition of
+// the partition key and clustering key for a table.
 func compileMetadata(
 	protoVersion int,
 	keyspace *KeyspaceMetadata,
@@ -178,8 +186,11 @@ func compileMetadata(
 	}
 }
 
-// V1 protocol does not return as much column metadata as V2+ so determining
-// PartitionKey and ClusterColumns is more complex
+// Compiles derived information from TableMetadata which have had
+// ColumnMetadata added already. V1 protocol does not return as much
+// column metadata as V2+ (because V1 doesn't support the "type" column in the
+// system.schema_columns table) so determining PartitionKey and ClusterColumns
+// is more complex.
 func compileV1Metadata(tables []TableMetadata) {
 	for i := range tables {
 		table := &tables[i]
@@ -308,6 +319,7 @@ func compileV2Metadata(tables []TableMetadata) {
 	}
 }
 
+// returns the count of coluns with the given "kind" value.
 func countColumnsOfKind(columns map[string]*ColumnMetadata, kind string) int {
 	count := 0
 	for _, column := range columns {
@@ -318,7 +330,7 @@ func countColumnsOfKind(columns map[string]*ColumnMetadata, kind string) int {
 	return count
 }
 
-// query only for the keyspace metadata for the specified keyspace
+// query only for the keyspace metadata for the specified keyspace from system.schema_keyspace
 func getKeyspaceMetadata(
 	session *Session,
 	keyspaceName string,
@@ -358,7 +370,7 @@ func getKeyspaceMetadata(
 	return keyspace, nil
 }
 
-// query for only the table metadata in the specified keyspace
+// query for only the table metadata in the specified keyspace from system.schema_columnfamilies
 func getTableMetadata(
 	session *Session,
 	keyspaceName string,
@@ -437,7 +449,7 @@ func getTableMetadata(
 	return tables, nil
 }
 
-// query for only the table metadata in the specified keyspace
+// query for only the column metadata in the specified keyspace from system.schema_columns
 func getColumnMetadata(
 	session *Session,
 	keyspaceName string,

+ 159 - 34
metadata_test.go

@@ -9,6 +9,8 @@ import (
 	"testing"
 )
 
+// Tests V1 and V2 metadata "compilation" from example data which might be returned
+// from metadata schema queries (see getKeyspaceMetadata, getTableMetadata, and getColumnMetadata)
 func TestCompileMetadata(t *testing.T) {
 	// V1 tests - these are all based on real examples from the integration test ccm cluster
 	keyspace := &KeyspaceMetadata{
@@ -53,7 +55,7 @@ func TestCompileMetadata(t *testing.T) {
 			Keyspace:         "V1Keyspace",
 			Name:             "IndexInfo",
 			KeyValidator:     "org.apache.cassandra.db.marshal.UTF8Type",
-			Comparator:       "org.apache.cassandra.db.marshal.UTF8Type",
+			Comparator:       "org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.UTF8Type)",
 			DefaultValidator: "org.apache.cassandra.db.marshal.BytesType",
 			KeyAliases:       []string{"table_name"},
 			ColumnAliases:    []string{"index_name"},
@@ -70,6 +72,17 @@ func TestCompileMetadata(t *testing.T) {
 			ColumnAliases:    []string{"revid"},
 			ValueAlias:       "",
 		},
+		TableMetadata{
+			// This is a made up example with multiple unnamed aliases
+			Keyspace:         "V1Keyspace",
+			Name:             "no_names",
+			KeyValidator:     "org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UUIDType,org.apache.cassandra.db.marshal.UUIDType)",
+			Comparator:       "org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.Int32Type)",
+			DefaultValidator: "org.apache.cassandra.db.marshal.BytesType",
+			KeyAliases:       []string{},
+			ColumnAliases:    []string{},
+			ValueAlias:       "",
+		},
 	}
 	columns := []ColumnMetadata{
 		// Here are the regular columns from the peers table for testing regular columns
@@ -182,7 +195,7 @@ func TestCompileMetadata(t *testing.T) {
 						&ColumnMetadata{
 							Name:  "index_name",
 							Type:  NativeType{typ: TypeVarchar},
-							Order: ASC,
+							Order: DESC,
 						},
 					},
 					Columns: map[string]*ColumnMetadata{
@@ -192,9 +205,10 @@ func TestCompileMetadata(t *testing.T) {
 							Kind: PARTITION_KEY,
 						},
 						"index_name": &ColumnMetadata{
-							Name: "index_name",
-							Type: NativeType{typ: TypeVarchar},
-							Kind: CLUSTERING_KEY,
+							Name:  "index_name",
+							Type:  NativeType{typ: TypeVarchar},
+							Order: DESC,
+							Kind:  CLUSTERING_KEY,
 						},
 						"value": &ColumnMetadata{
 							Name: "value",
@@ -230,6 +244,70 @@ func TestCompileMetadata(t *testing.T) {
 						},
 					},
 				},
+				"no_names": &TableMetadata{
+					PartitionKey: []*ColumnMetadata{
+						&ColumnMetadata{
+							Name: "key",
+							Type: NativeType{typ: TypeUUID},
+						},
+						&ColumnMetadata{
+							Name: "key2",
+							Type: NativeType{typ: TypeUUID},
+						},
+					},
+					ClusteringColumns: []*ColumnMetadata{
+						&ColumnMetadata{
+							Name:  "column",
+							Type:  NativeType{typ: TypeInt},
+							Order: ASC,
+						},
+						&ColumnMetadata{
+							Name:  "column2",
+							Type:  NativeType{typ: TypeInt},
+							Order: ASC,
+						},
+						&ColumnMetadata{
+							Name:  "column3",
+							Type:  NativeType{typ: TypeInt},
+							Order: ASC,
+						},
+					},
+					Columns: map[string]*ColumnMetadata{
+						"key": &ColumnMetadata{
+							Name: "key",
+							Type: NativeType{typ: TypeUUID},
+							Kind: PARTITION_KEY,
+						},
+						"key2": &ColumnMetadata{
+							Name: "key2",
+							Type: NativeType{typ: TypeUUID},
+							Kind: PARTITION_KEY,
+						},
+						"column": &ColumnMetadata{
+							Name:  "column",
+							Type:  NativeType{typ: TypeInt},
+							Order: ASC,
+							Kind:  CLUSTERING_KEY,
+						},
+						"column2": &ColumnMetadata{
+							Name:  "column2",
+							Type:  NativeType{typ: TypeInt},
+							Order: ASC,
+							Kind:  CLUSTERING_KEY,
+						},
+						"column3": &ColumnMetadata{
+							Name:  "column3",
+							Type:  NativeType{typ: TypeInt},
+							Order: ASC,
+							Kind:  CLUSTERING_KEY,
+						},
+						"value": &ColumnMetadata{
+							Name: "value",
+							Type: NativeType{typ: TypeBlob},
+							Kind: REGULAR,
+						},
+					},
+				},
 			},
 		},
 	)
@@ -250,30 +328,41 @@ func TestCompileMetadata(t *testing.T) {
 	}
 	columns = []ColumnMetadata{
 		ColumnMetadata{
-			Keyspace:  "V2Keyspace",
-			Table:     "Table1",
-			Name:      "Key1",
-			Kind:      PARTITION_KEY,
-			Validator: "org.apache.cassandra.db.marshal.UTF8Type",
+			Keyspace:       "V2Keyspace",
+			Table:          "Table1",
+			Name:           "Key1",
+			Kind:           PARTITION_KEY,
+			ComponentIndex: 0,
+			Validator:      "org.apache.cassandra.db.marshal.UTF8Type",
 		},
 		ColumnMetadata{
-			Keyspace:  "V2Keyspace",
-			Table:     "Table2",
-			Name:      "Column1",
-			Kind:      PARTITION_KEY,
-			Validator: "org.apache.cassandra.db.marshal.UTF8Type",
+			Keyspace:       "V2Keyspace",
+			Table:          "Table2",
+			Name:           "Column1",
+			Kind:           PARTITION_KEY,
+			ComponentIndex: 0,
+			Validator:      "org.apache.cassandra.db.marshal.UTF8Type",
 		},
 		ColumnMetadata{
-			Keyspace:  "V2Keyspace",
-			Table:     "Table2",
-			Name:      "Column2",
-			Kind:      CLUSTERING_KEY,
-			Validator: "org.apache.cassandra.db.marshal.UTF8Type",
+			Keyspace:       "V2Keyspace",
+			Table:          "Table2",
+			Name:           "Column2",
+			Kind:           CLUSTERING_KEY,
+			ComponentIndex: 0,
+			Validator:      "org.apache.cassandra.db.marshal.UTF8Type",
+		},
+		ColumnMetadata{
+			Keyspace:       "V2Keyspace",
+			Table:          "Table2",
+			Name:           "Column3",
+			Kind:           CLUSTERING_KEY,
+			ComponentIndex: 1,
+			Validator:      "org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.UTF8Type)",
 		},
 		ColumnMetadata{
 			Keyspace:  "V2Keyspace",
 			Table:     "Table2",
-			Name:      "Column3",
+			Name:      "Column4",
 			Kind:      REGULAR,
 			Validator: "org.apache.cassandra.db.marshal.UTF8Type",
 		},
@@ -310,8 +399,14 @@ func TestCompileMetadata(t *testing.T) {
 					},
 					ClusteringColumns: []*ColumnMetadata{
 						&ColumnMetadata{
-							Name: "Column2",
-							Type: NativeType{typ: TypeVarchar},
+							Name:  "Column2",
+							Type:  NativeType{typ: TypeVarchar},
+							Order: ASC,
+						},
+						&ColumnMetadata{
+							Name:  "Column3",
+							Type:  NativeType{typ: TypeVarchar},
+							Order: DESC,
 						},
 					},
 					Columns: map[string]*ColumnMetadata{
@@ -321,12 +416,19 @@ func TestCompileMetadata(t *testing.T) {
 							Kind: PARTITION_KEY,
 						},
 						"Column2": &ColumnMetadata{
-							Name: "Column2",
-							Type: NativeType{typ: TypeVarchar},
-							Kind: CLUSTERING_KEY,
+							Name:  "Column2",
+							Type:  NativeType{typ: TypeVarchar},
+							Order: ASC,
+							Kind:  CLUSTERING_KEY,
 						},
 						"Column3": &ColumnMetadata{
-							Name: "Column3",
+							Name:  "Column3",
+							Type:  NativeType{typ: TypeVarchar},
+							Order: DESC,
+							Kind:  CLUSTERING_KEY,
+						},
+						"Column4": &ColumnMetadata{
+							Name: "Column4",
 							Type: NativeType{typ: TypeVarchar},
 							Kind: REGULAR,
 						},
@@ -337,6 +439,7 @@ func TestCompileMetadata(t *testing.T) {
 	)
 }
 
+// Helper function for asserting that actual metadata returned was as expected
 func assertKeyspaceMetadata(t *testing.T, actual, expected *KeyspaceMetadata) {
 	if len(expected.Tables) != len(actual.Tables) {
 		t.Errorf("Expected len(%s.Tables) to be %v but was %v", expected.Name, len(expected.Tables), len(actual.Tables))
@@ -379,6 +482,9 @@ func assertKeyspaceMetadata(t *testing.T, actual, expected *KeyspaceMetadata) {
 				t.Errorf("Expected len(%s.Tables[%s].ClusteringColumns) to be %v but was %v", expected.Name, keyT, len(et.ClusteringColumns), len(at.ClusteringColumns))
 			} else {
 				for i := range et.ClusteringColumns {
+					if at.ClusteringColumns[i] == nil {
+						t.Fatalf("Unexpected nil value: %s.Tables[%s].ClusteringColumns[%d]", expected.Name, keyT, i)
+					}
 					if et.ClusteringColumns[i].Name != at.ClusteringColumns[i].Name {
 						t.Errorf("Expected %s.Tables[%s].ClusteringColumns[%d].Name to be '%v' but was '%v'", expected.Name, keyT, i, et.ClusteringColumns[i].Name, at.ClusteringColumns[i].Name)
 					}
@@ -445,6 +551,7 @@ func assertKeyspaceMetadata(t *testing.T, actual, expected *KeyspaceMetadata) {
 	}
 }
 
+// Tests the cassandra type definition parser
 func TestTypeParser(t *testing.T) {
 	// native type
 	assertParseNonCompositeType(
@@ -470,10 +577,20 @@ func TestTypeParser(t *testing.T) {
 		},
 	)
 
+	// list
+	assertParseNonCompositeType(
+		t,
+		"org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.TimeUUIDType)",
+		assertTypeInfo{
+			Type: TypeList,
+			Elem: &assertTypeInfo{Type: TypeTimeUUID},
+		},
+	)
+
 	// map
 	assertParseNonCompositeType(
 		t,
-		"org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UUIDType,org.apache.cassandra.db.marshal.BytesType)",
+		" org.apache.cassandra.db.marshal.MapType( org.apache.cassandra.db.marshal.UUIDType , org.apache.cassandra.db.marshal.BytesType ) ",
 		assertTypeInfo{
 			Type: TypeMap,
 			Key:  &assertTypeInfo{Type: TypeUUID},
@@ -482,6 +599,11 @@ func TestTypeParser(t *testing.T) {
 	)
 
 	// custom
+	assertParseNonCompositeType(
+		t,
+		"org.apache.cassandra.db.marshal.UserType(sandbox,61646472657373,737472656574:org.apache.cassandra.db.marshal.UTF8Type,63697479:org.apache.cassandra.db.marshal.UTF8Type,7a6970:org.apache.cassandra.db.marshal.Int32Type)",
+		assertTypeInfo{Type: TypeCustom, Custom: "org.apache.cassandra.db.marshal.UserType(sandbox,61646472657373,737472656574:org.apache.cassandra.db.marshal.UTF8Type,63697479:org.apache.cassandra.db.marshal.UTF8Type,7a6970:org.apache.cassandra.db.marshal.Int32Type)"},
+	)
 	assertParseNonCompositeType(
 		t,
 		"org.apache.cassandra.db.marshal.DynamicCompositeType(u=>org.apache.cassandra.db.marshal.UUIDType,d=>org.apache.cassandra.db.marshal.DateType,t=>org.apache.cassandra.db.marshal.TimeUUIDType,b=>org.apache.cassandra.db.marshal.BytesType,s=>org.apache.cassandra.db.marshal.UTF8Type,B=>org.apache.cassandra.db.marshal.BooleanType,a=>org.apache.cassandra.db.marshal.AsciiType,l=>org.apache.cassandra.db.marshal.LongType,i=>org.apache.cassandra.db.marshal.IntegerType,x=>org.apache.cassandra.db.marshal.LexicalUUIDType)",
@@ -499,9 +621,9 @@ func TestTypeParser(t *testing.T) {
 	)
 	assertParseCompositeType(
 		t,
-		"org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.DateType,org.apache.cassandra.db.marshal.UTF8Type)",
+		"org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.DateType),org.apache.cassandra.db.marshal.UTF8Type)",
 		[]assertTypeInfo{
-			assertTypeInfo{Type: TypeTimestamp},
+			assertTypeInfo{Type: TypeTimestamp, Reversed: true},
 			assertTypeInfo{Type: TypeVarchar},
 		},
 		nil,
@@ -522,10 +644,7 @@ func TestTypeParser(t *testing.T) {
 	)
 }
 
-//---------------------------------------
-// some code to assert the parser result
-//---------------------------------------
-
+// expected data holder
 type assertTypeInfo struct {
 	Type     Type
 	Reversed bool
@@ -534,6 +653,8 @@ type assertTypeInfo struct {
 	Custom   string
 }
 
+// Helper function for asserting that the type parser returns the expected
+// results for the given definition
 func assertParseNonCompositeType(
 	t *testing.T,
 	def string,
@@ -561,6 +682,8 @@ func assertParseNonCompositeType(
 	}
 }
 
+// Helper function for asserting that the type parser returns the expected
+// results for the given definition
 func assertParseCompositeType(
 	t *testing.T,
 	def string,
@@ -612,6 +735,8 @@ func assertParseCompositeType(
 	}
 }
 
+// Helper function for asserting that the type parser returns the expected
+// results for the given definition
 func assertParseNonCompositeTypes(
 	t *testing.T,
 	context string,

+ 5 - 1
policies.go

@@ -87,12 +87,16 @@ func (r *roundRobinHostPolicy) Pick(qry *Query) NextHost {
 	// to the number of hosts known to this policy
 	var i uint32 = 0
 	return func() *HostInfo {
+		if len(r.hosts) == 0 {
+			return nil
+		}
+
 		var host *HostInfo
 		r.mu.RLock()
 		// always increment pos to evenly distribute traffic in case of
 		// failures
 		pos := atomic.AddUint32(&r.pos, 1)
-		if len(r.hosts) > 0 && int(i) < len(r.hosts) {
+		if int(i) < len(r.hosts) {
 			host = &r.hosts[(pos)%uint32(len(r.hosts))]
 			i++
 		}

+ 46 - 9
policies_test.go

@@ -6,6 +6,7 @@ package gocql
 
 import "testing"
 
+// Tests of the round-robin host selection policy implementation
 func TestRoundRobinHostPolicy(t *testing.T) {
 	policy := NewRoundRobinHostPolicy()
 
@@ -42,27 +43,63 @@ func TestRoundRobinHostPolicy(t *testing.T) {
 	}
 }
 
+// Tests of the token-aware host selection policy implementation with a
+// round-robin host selection policy fallback.
 func TestTokenAwareHostPolicy(t *testing.T) {
 	policy := NewTokenAwareHostPolicy(NewRoundRobinHostPolicy())
 
-	hosts := []HostInfo{
-		HostInfo{HostId: "0", Peer: "0", Tokens: []string{"00"}},
-		HostInfo{HostId: "1", Peer: "1", Tokens: []string{"25"}},
-		HostInfo{HostId: "2", Peer: "2", Tokens: []string{"50"}},
-		HostInfo{HostId: "3", Peer: "3", Tokens: []string{"75"}},
+	query := &Query{}
+
+	iter := policy.Pick(nil)
+	if iter == nil {
+		t.Fatal("host iterator was nil")
+	}
+	actual := iter()
+	if actual != nil {
+		t.Fatalf("expected nil from iterator, but was %v", actual)
 	}
 
+	// set the hosts
+	hosts := []HostInfo{
+		HostInfo{Peer: "0", Tokens: []string{"00"}},
+		HostInfo{Peer: "1", Tokens: []string{"25"}},
+		HostInfo{Peer: "2", Tokens: []string{"50"}},
+		HostInfo{Peer: "3", Tokens: []string{"75"}},
+	}
 	policy.SetHosts(hosts)
-	policy.SetPartitioner("OrderedPartitioner")
 
-	query := &Query{}
+	// the token ring is not setup without the partitioner, but the fallback
+	// should work
+	if actual := policy.Pick(nil)(); actual.Peer != "1" {
+		t.Errorf("Expected peer 1 but was %s", actual.Peer)
+	}
+
 	query.RoutingKey([]byte("30"))
+	if actual := policy.Pick(query)(); actual.Peer != "2" {
+		t.Errorf("Expected peer 2 but was %s", actual.Peer)
+	}
+
+	policy.SetPartitioner("OrderedPartitioner")
 
-	if actual := policy.Pick(query)(); actual != &hosts[2] {
-		t.Errorf("Expected hosts[2] but was hosts[%s]", actual.HostId)
+	// now the token ring is configured
+	query.RoutingKey([]byte("20"))
+	iter = policy.Pick(query)
+	if actual := iter(); actual.Peer != "1" {
+		t.Errorf("Expected peer 1 but was %s", actual.Peer)
+	}
+	// rest are round robin
+	if actual := iter(); actual.Peer != "3" {
+		t.Errorf("Expected peer 3 but was %s", actual.Peer)
+	}
+	if actual := iter(); actual.Peer != "0" {
+		t.Errorf("Expected peer 0 but was %s", actual.Peer)
+	}
+	if actual := iter(); actual.Peer != "2" {
+		t.Errorf("Expected peer 2 but was %s", actual.Peer)
 	}
 }
 
+// Tests of the round-robin connection selection policy implementation
 func TestRoundRobinConnPolicy(t *testing.T) {
 	policy := NewRoundRobinConnPolicy()
 

+ 29 - 11
token.go

@@ -17,6 +17,7 @@ import (
 
 // a token partitioner
 type partitioner interface {
+	Name() string
 	Hash([]byte) token
 	ParseString(string) token
 }
@@ -31,6 +32,10 @@ type token interface {
 type murmur3Partitioner struct{}
 type murmur3Token int64
 
+func (p murmur3Partitioner) Name() string {
+	return "Murmur3Partitioner"
+}
+
 func (p murmur3Partitioner) Hash(partitionKey []byte) token {
 	h1 := murmur3H1(partitionKey)
 	return murmur3Token(int64(h1))
@@ -183,30 +188,38 @@ func (m murmur3Token) Less(token token) bool {
 }
 
 // order preserving partitioner and token
-type orderPreservingPartitioner struct{}
-type orderPreservingToken []byte
+type orderedPartitioner struct{}
+type orderedToken []byte
+
+func (p orderedPartitioner) Name() string {
+	return "OrderedPartitioner"
+}
 
-func (p orderPreservingPartitioner) Hash(partitionKey []byte) token {
+func (p orderedPartitioner) Hash(partitionKey []byte) token {
 	// the partition key is the token
-	return orderPreservingToken(partitionKey)
+	return orderedToken(partitionKey)
 }
 
-func (p orderPreservingPartitioner) ParseString(str string) token {
-	return orderPreservingToken([]byte(str))
+func (p orderedPartitioner) ParseString(str string) token {
+	return orderedToken([]byte(str))
 }
 
-func (o orderPreservingToken) String() string {
+func (o orderedToken) String() string {
 	return string([]byte(o))
 }
 
-func (o orderPreservingToken) Less(token token) bool {
-	return -1 == bytes.Compare(o, token.(orderPreservingToken))
+func (o orderedToken) Less(token token) bool {
+	return -1 == bytes.Compare(o, token.(orderedToken))
 }
 
 // random partitioner and token
 type randomPartitioner struct{}
 type randomToken big.Int
 
+func (r randomPartitioner) Name() string {
+	return "RandomPartitioner"
+}
+
 func (p randomPartitioner) Hash(partitionKey []byte) token {
 	hash := md5.New()
 	sum := hash.Sum(partitionKey)
@@ -248,7 +261,7 @@ func newTokenRing(partitioner string, hosts []HostInfo) (*tokenRing, error) {
 	if strings.HasSuffix(partitioner, "Murmur3Partitioner") {
 		tokenRing.partitioner = murmur3Partitioner{}
 	} else if strings.HasSuffix(partitioner, "OrderedPartitioner") {
-		tokenRing.partitioner = orderPreservingPartitioner{}
+		tokenRing.partitioner = orderedPartitioner{}
 	} else if strings.HasSuffix(partitioner, "RandomPartitioner") {
 		tokenRing.partitioner = randomPartitioner{}
 	} else {
@@ -283,8 +296,13 @@ func (t *tokenRing) Swap(i, j int) {
 }
 
 func (t *tokenRing) String() string {
+
 	buf := &bytes.Buffer{}
-	buf.WriteString("TokenRing={")
+	buf.WriteString("TokenRing(")
+	if t.partitioner != nil {
+		buf.WriteString(t.partitioner.Name())
+	}
+	buf.WriteString("){")
 	sep := ""
 	for i := range t.tokens {
 		buf.WriteString(sep)

+ 315 - 30
token_test.go

@@ -5,23 +5,56 @@
 package gocql
 
 import (
+	"bytes"
 	"math/big"
+	"sort"
 	"strconv"
 	"testing"
 )
 
+// Test the implementation of murmur3
 func TestMurmur3H1(t *testing.T) {
-	assertMurmur3H1(t, []byte{}, 0x000000000000000)
-	assertMurmur3H1(t, []byte{0}, 0x4610abe56eff5cb5)
-	assertMurmur3H1(t, []byte{0, 1}, 0x7cb3f5c58dab264c)
-	assertMurmur3H1(t, []byte{0, 1, 2}, 0xb872a12fef53e6be)
-	assertMurmur3H1(t, []byte{0, 1, 2, 3}, 0xe1c594ae0ddfaf10)
+	// these examples are based on adding a index number to a sample string in
+	// a loop. The expected values were generated by the java datastax murmur3
+	// implementation. The number of examples here of increasing lengths ensure
+	// test coverage of all tail-length branches in the murmur3 algorithm
+	seriesExpected := [...]uint64{
+		0x0000000000000000, // ""
+		0x2ac9debed546a380, // "0"
+		0x649e4eaa7fc1708e, // "01"
+		0xce68f60d7c353bdb, // "012"
+		0x0f95757ce7f38254, // "0123"
+		0x0f04e459497f3fc1, // "01234"
+		0x88c0a92586be0a27, // "012345"
+		0x13eb9fb82606f7a6, // "0123456"
+		0x8236039b7387354d, // "01234567"
+		0x4c1e87519fe738ba, // "012345678"
+		0x3f9652ac3effeb24, // "0123456789"
+		0x3f33760ded9006c6, // "01234567890"
+		0xaed70a6631854cb1, // "012345678901"
+		0x8a299a8f8e0e2da7, // "0123456789012"
+		0x624b675c779249a6, // "01234567890123"
+		0xa4b203bb1d90b9a3, // "012345678901234"
+		0xa3293ad698ecb99a, // "0123456789012345"
+		0xbc740023dbd50048, // "01234567890123456"
+		0x3fe5ab9837d25cdd, // "012345678901234567"
+		0x2d0338c1ca87d132, // "0123456789012345678"
+	}
+	sample := ""
+	for i, expected := range seriesExpected {
+		assertMurmur3H1(t, []byte(sample), expected)
+
+		sample = sample + strconv.Itoa(i%10)
+	}
+
+	// Here are some test examples from other driver implementations
 	assertMurmur3H1(t, []byte("hello"), 0xcbd8a7b341bd9b02)
 	assertMurmur3H1(t, []byte("hello, world"), 0x342fac623a5ebc8e)
 	assertMurmur3H1(t, []byte("19 Jan 2038 at 3:14:07 AM"), 0xb89e5988b737affc)
 	assertMurmur3H1(t, []byte("The quick brown fox jumps over the lazy dog."), 0xcd99481f9ee902c9)
 }
 
+// helper function for testing the murmur3 implementation
 func assertMurmur3H1(t *testing.T, data []byte, expected uint64) {
 	actual := murmur3H1(data)
 	if actual != expected {
@@ -29,6 +62,7 @@ func assertMurmur3H1(t *testing.T, data []byte, expected uint64) {
 	}
 }
 
+// Benchmark of the performance of the murmur3 implementation
 func BenchmarkMurmur3H1(b *testing.B) {
 	var h1 uint64
 	var data [1024]byte
@@ -42,6 +76,7 @@ func BenchmarkMurmur3H1(b *testing.B) {
 	}
 }
 
+// Tests of the murmur3Patitioner
 func TestMurmur3Partitioner(t *testing.T) {
 	token := murmur3Partitioner{}.ParseString("-1053604476080545076")
 
@@ -58,6 +93,7 @@ func TestMurmur3Partitioner(t *testing.T) {
 	}
 }
 
+// Tests of the murmur3Token
 func TestMurmur3Token(t *testing.T) {
 	if murmur3Token(42).Less(murmur3Token(42)) {
 		t.Errorf("Expected Less to return false, but was true")
@@ -70,38 +106,66 @@ func TestMurmur3Token(t *testing.T) {
 	}
 }
 
-func TestOrderPreservingPartitioner(t *testing.T) {
+// Tests of the orderedPartitioner
+func TestOrderedPartitioner(t *testing.T) {
 	// at least verify that the partitioner
 	// doesn't return nil
+	p := orderedPartitioner{}
 	pk, _ := marshalInt(nil, 1)
-	token := orderPreservingPartitioner{}.Hash(pk)
+	token := p.Hash(pk)
 	if token == nil {
 		t.Fatal("token was nil")
 	}
+
+	str := token.String()
+	parsedToken := p.ParseString(str)
+
+	if !bytes.Equal([]byte(token.(orderedToken)), []byte(parsedToken.(orderedToken))) {
+		t.Errorf("Failed to convert to and from a string %s expected %x but was %x",
+			str,
+			[]byte(token.(orderedToken)),
+			[]byte(parsedToken.(orderedToken)),
+		)
+	}
 }
 
-func TestOrderPreservingToken(t *testing.T) {
-	if orderPreservingToken([]byte{0, 0, 4, 2}).Less(orderPreservingToken([]byte{0, 0, 4, 2})) {
+// Tests of the orderedToken
+func TestOrderedToken(t *testing.T) {
+	if orderedToken([]byte{0, 0, 4, 2}).Less(orderedToken([]byte{0, 0, 4, 2})) {
 		t.Errorf("Expected Less to return false, but was true")
 	}
-	if !orderPreservingToken([]byte{0, 0, 3}).Less(orderPreservingToken([]byte{0, 0, 4, 2})) {
+	if !orderedToken([]byte{0, 0, 3}).Less(orderedToken([]byte{0, 0, 4, 2})) {
 		t.Errorf("Expected Less to return true, but was false")
 	}
-	if orderPreservingToken([]byte{0, 0, 4, 2}).Less(orderPreservingToken([]byte{0, 0, 3})) {
+	if orderedToken([]byte{0, 0, 4, 2}).Less(orderedToken([]byte{0, 0, 3})) {
 		t.Errorf("Expected Less to return false, but was true")
 	}
 }
 
+// Tests of the randomPartitioner
 func TestRandomPartitioner(t *testing.T) {
 	// at least verify that the partitioner
 	// doesn't return nil
+	p := randomPartitioner{}
 	pk, _ := marshalInt(nil, 1)
-	token := randomPartitioner{}.Hash(pk)
+	token := p.Hash(pk)
 	if token == nil {
 		t.Fatal("token was nil")
 	}
+
+	str := token.String()
+	parsedToken := p.ParseString(str)
+
+	if (*big.Int)(token.(*randomToken)).Cmp((*big.Int)(parsedToken.(*randomToken))) != 0 {
+		t.Errorf("Failed to convert to and from a string %s expected %v but was %v",
+			str,
+			token,
+			parsedToken,
+		)
+	}
 }
 
+// Tests of the randomToken
 func TestRandomToken(t *testing.T) {
 	if ((*randomToken)(big.NewInt(42))).Less((*randomToken)(big.NewInt(42))) {
 		t.Errorf("Expected Less to return false, but was true")
@@ -124,66 +188,287 @@ func (i intToken) Less(token token) bool {
 	return i < token.(intToken)
 }
 
+// Test of the token ring implementation based on example at the start of this
+// page of documentation:
+// http://www.datastax.com/docs/0.8/cluster_architecture/partitioning
 func TestIntTokenRing(t *testing.T) {
-	// test based on example at the start of this page of documentation:
-	// http://www.datastax.com/docs/0.8/cluster_architecture/partitioning
 	host0 := &HostInfo{}
 	host25 := &HostInfo{}
 	host50 := &HostInfo{}
 	host75 := &HostInfo{}
-	tokenRing := &tokenRing{
+	ring := &tokenRing{
 		partitioner: nil,
+		// these tokens and hosts are out of order to test sorting
 		tokens: []token{
 			intToken(0),
-			intToken(25),
 			intToken(50),
 			intToken(75),
+			intToken(25),
 		},
 		hosts: []*HostInfo{
 			host0,
-			host25,
 			host50,
 			host75,
+			host25,
 		},
 	}
 
-	if tokenRing.GetHostForToken(intToken(0)) != host0 {
+	sort.Sort(ring)
+
+	if ring.GetHostForToken(intToken(0)) != host0 {
 		t.Error("Expected host 0 for token 0")
 	}
-	if tokenRing.GetHostForToken(intToken(1)) != host25 {
+	if ring.GetHostForToken(intToken(1)) != host25 {
 		t.Error("Expected host 25 for token 1")
 	}
-	if tokenRing.GetHostForToken(intToken(24)) != host25 {
+	if ring.GetHostForToken(intToken(24)) != host25 {
 		t.Error("Expected host 25 for token 24")
 	}
-	if tokenRing.GetHostForToken(intToken(25)) != host25 {
+	if ring.GetHostForToken(intToken(25)) != host25 {
 		t.Error("Expected host 25 for token 25")
 	}
-	if tokenRing.GetHostForToken(intToken(26)) != host50 {
+	if ring.GetHostForToken(intToken(26)) != host50 {
 		t.Error("Expected host 50 for token 26")
 	}
-	if tokenRing.GetHostForToken(intToken(49)) != host50 {
+	if ring.GetHostForToken(intToken(49)) != host50 {
 		t.Error("Expected host 50 for token 49")
 	}
-	if tokenRing.GetHostForToken(intToken(50)) != host50 {
+	if ring.GetHostForToken(intToken(50)) != host50 {
 		t.Error("Expected host 50 for token 50")
 	}
-	if tokenRing.GetHostForToken(intToken(51)) != host75 {
+	if ring.GetHostForToken(intToken(51)) != host75 {
 		t.Error("Expected host 75 for token 51")
 	}
-	if tokenRing.GetHostForToken(intToken(74)) != host75 {
+	if ring.GetHostForToken(intToken(74)) != host75 {
 		t.Error("Expected host 75 for token 74")
 	}
-	if tokenRing.GetHostForToken(intToken(75)) != host75 {
+	if ring.GetHostForToken(intToken(75)) != host75 {
 		t.Error("Expected host 75 for token 75")
 	}
-	if tokenRing.GetHostForToken(intToken(76)) != host0 {
+	if ring.GetHostForToken(intToken(76)) != host0 {
 		t.Error("Expected host 0 for token 76")
 	}
-	if tokenRing.GetHostForToken(intToken(99)) != host0 {
+	if ring.GetHostForToken(intToken(99)) != host0 {
 		t.Error("Expected host 0 for token 99")
 	}
-	if tokenRing.GetHostForToken(intToken(100)) != host0 {
+	if ring.GetHostForToken(intToken(100)) != host0 {
 		t.Error("Expected host 0 for token 100")
 	}
 }
+
+// Test for the behavior of a nil pointer to tokenRing
+func TestNilTokenRing(t *testing.T) {
+	var ring *tokenRing = nil
+
+	if ring.GetHostForToken(nil) != nil {
+		t.Error("Expected nil for nil token ring")
+	}
+	if ring.GetHostForPartitionKey(nil) != nil {
+		t.Error("Expected nil for nil token ring")
+	}
+}
+
+// Test of the recognition of the partitioner class
+func TestUnknownTokenRing(t *testing.T) {
+	_, err := newTokenRing("UnknownPartitioner", nil)
+	if err == nil {
+		t.Error("Expected error for unknown partitioner value, but was nil")
+	}
+}
+
+// Test of the tokenRing with the Murmur3Partitioner
+func TestMurmur3TokenRing(t *testing.T) {
+	// Note, strings are parsed directly to int64, they are not murmur3 hashed
+	var hosts []HostInfo = []HostInfo{
+		HostInfo{
+			Peer:   "0",
+			Tokens: []string{"0"},
+		},
+		HostInfo{
+			Peer:   "1",
+			Tokens: []string{"25"},
+		},
+		HostInfo{
+			Peer:   "2",
+			Tokens: []string{"50"},
+		},
+		HostInfo{
+			Peer:   "3",
+			Tokens: []string{"75"},
+		},
+	}
+	ring, err := newTokenRing("Murmur3Partitioner", hosts)
+	if err != nil {
+		t.Fatalf("Failed to create token ring due to error: %v", err)
+	}
+
+	p := murmur3Partitioner{}
+
+	var actual *HostInfo
+	actual = ring.GetHostForToken(p.ParseString("0"))
+	if actual.Peer != "0" {
+		t.Errorf("Expected peer 0 for token \"0\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("25"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"25\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("50"))
+	if actual.Peer != "2" {
+		t.Errorf("Expected peer 2 for token \"50\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("75"))
+	if actual.Peer != "3" {
+		t.Errorf("Expected peer 3 for token \"01\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("12"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"12\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("24324545443332"))
+	if actual.Peer != "0" {
+		t.Errorf("Expected peer 0 for token \"24324545443332\", but was %s", actual.Peer)
+	}
+}
+
+// Test of the tokenRing with the OrderedPartitioner
+func TestOrderedTokenRing(t *testing.T) {
+	// Tokens here more or less are similar layout to the int tokens above due
+	// to each numeric character translating to a consistently offset byte.
+	var hosts []HostInfo = []HostInfo{
+		HostInfo{
+			Peer: "0",
+			Tokens: []string{
+				"00",
+			},
+		},
+		HostInfo{
+			Peer: "1",
+			Tokens: []string{
+				"25",
+			},
+		},
+		HostInfo{
+			Peer: "2",
+			Tokens: []string{
+				"50",
+			},
+		},
+		HostInfo{
+			Peer: "3",
+			Tokens: []string{
+				"75",
+			},
+		},
+	}
+	ring, err := newTokenRing("OrderedPartitioner", hosts)
+	if err != nil {
+		t.Fatalf("Failed to create token ring due to error: %v", err)
+	}
+
+	p := orderedPartitioner{}
+
+	var actual *HostInfo
+	actual = ring.GetHostForToken(p.ParseString("0"))
+	if actual.Peer != "0" {
+		t.Errorf("Expected peer 0 for token \"0\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("25"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"25\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("50"))
+	if actual.Peer != "2" {
+		t.Errorf("Expected peer 2 for token \"50\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("75"))
+	if actual.Peer != "3" {
+		t.Errorf("Expected peer 3 for token \"01\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("12"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"12\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("24324545443332"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"24324545443332\", but was %s", actual.Peer)
+	}
+}
+
+// Test of the tokenRing with the RandomPartitioner
+func TestRandomTokenRing(t *testing.T) {
+	// String tokens are parsed into big.Int in base 10
+	var hosts []HostInfo = []HostInfo{
+		HostInfo{
+			Peer: "0",
+			Tokens: []string{
+				"00",
+			},
+		},
+		HostInfo{
+			Peer: "1",
+			Tokens: []string{
+				"25",
+			},
+		},
+		HostInfo{
+			Peer: "2",
+			Tokens: []string{
+				"50",
+			},
+		},
+		HostInfo{
+			Peer: "3",
+			Tokens: []string{
+				"75",
+			},
+		},
+	}
+	ring, err := newTokenRing("RandomPartitioner", hosts)
+	if err != nil {
+		t.Fatalf("Failed to create token ring due to error: %v", err)
+	}
+
+	p := randomPartitioner{}
+
+	var actual *HostInfo
+	actual = ring.GetHostForToken(p.ParseString("0"))
+	if actual.Peer != "0" {
+		t.Errorf("Expected peer 0 for token \"0\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("25"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"25\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("50"))
+	if actual.Peer != "2" {
+		t.Errorf("Expected peer 2 for token \"50\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("75"))
+	if actual.Peer != "3" {
+		t.Errorf("Expected peer 3 for token \"01\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("12"))
+	if actual.Peer != "1" {
+		t.Errorf("Expected peer 1 for token \"12\", but was %s", actual.Peer)
+	}
+
+	actual = ring.GetHostForToken(p.ParseString("24324545443332"))
+	if actual.Peer != "0" {
+		t.Errorf("Expected peer 0 for token \"24324545443332\", but was %s", actual.Peer)
+	}
+}