This is an automated email from the ASF dual-hosted git repository.

joaoreis pushed a commit to branch trunk
in repository https://gitbox.apache.org/repos/asf/cassandra-gocql-driver.git


The following commit(s) were added to refs/heads/trunk by this push:
     new 61c0b48a remove proto 1 and 2 support
61c0b48a is described below

commit 61c0b48aac0d6481b8294ae88db75582bce955df
Author: Dmitry Kropachev <dmitry.kropac...@gmail.com>
AuthorDate: Thu May 29 04:23:19 2025 -0400

    remove proto 1 and 2 support
    
    You can't find servers that support only proto 1 or 2.
    And there is no technical benefits to restrict proto to these versions.
    So, we can easily remove code that is taking care of these versions.
    
    Patch by dkropachev; reviewed by joao-r-reis for CASSGO-75
---
 CHANGELOG.md      |   4 +
 batch_test.go     |   8 -
 cassandra_test.go |  24 +--
 conn.go           |  11 +-
 conn_test.go      |  16 +-
 frame.go          | 254 ++++++++----------------
 frame_test.go     |  10 +-
 marshal.go        |  69 +++----
 marshal_test.go   | 573 ++++++++++++++++++++++++------------------------------
 metadata.go       | 181 +----------------
 metadata_test.go  | 305 +----------------------------
 tuple_test.go     |  30 +--
 udt_test.go       |  40 ----
 13 files changed, 393 insertions(+), 1132 deletions(-)

diff --git a/CHANGELOG.md b/CHANGELOG.md
index 99f1defb..7d0d5c7e 100644
--- a/CHANGELOG.md
+++ b/CHANGELOG.md
@@ -7,6 +7,10 @@ and this project adheres to [Semantic 
Versioning](https://semver.org/spec/v2.0.0
 
 ## [Unreleased]
 
+### Removed
+
+- Drop support for old CQL protocol versions: 1 and 2 (CASSGO-75)
+
 ### Added
 
 - Support vector type 
[CASSGO-11](https://issues.apache.org/jira/browse/CASSGO-11)
diff --git a/batch_test.go b/batch_test.go
index 3a8a7e75..47adff83 100644
--- a/batch_test.go
+++ b/batch_test.go
@@ -40,10 +40,6 @@ func TestBatch_Errors(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion2 {
-               t.Skip("atomic batches not supported. Please use Cassandra >= 
2.0")
-       }
-
        if err := createTable(session, `CREATE TABLE gocql_test.batch_errors 
(id int primary key, val inet)`); err != nil {
                t.Fatal(err)
        }
@@ -59,10 +55,6 @@ func TestBatch_WithTimestamp(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("Batch timestamps are only available on protocol >= 3")
-       }
-
        if err := createTable(session, `CREATE TABLE gocql_test.batch_ts (id 
int primary key, val text)`); err != nil {
                t.Fatal(err)
        }
diff --git a/cassandra_test.go b/cassandra_test.go
index 039fcc1b..1a9b69fe 100644
--- a/cassandra_test.go
+++ b/cassandra_test.go
@@ -2284,25 +2284,7 @@ func TestGetTableMetadata(t *testing.T) {
        if testTable == nil {
                t.Fatal("Expected table metadata for name 
'test_table_metadata'")
        }
-       if session.cfg.ProtoVersion == protoVersion1 {
-               if testTable.KeyValidator != 
"org.apache.cassandra.db.marshal.Int32Type" {
-                       t.Errorf("Expected test_table_metadata key validator to 
be 'org.apache.cassandra.db.marshal.Int32Type' but was '%s'", 
testTable.KeyValidator)
-               }
-               if testTable.Comparator != 
"org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.UTF8Type)"
 {
-                       t.Errorf("Expected test_table_metadata key validator to 
be 
'org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.UTF8Type)'
 but was '%s'", testTable.Comparator)
-               }
-               if testTable.DefaultValidator != 
"org.apache.cassandra.db.marshal.BytesType" {
-                       t.Errorf("Expected test_table_metadata key validator to 
be 'org.apache.cassandra.db.marshal.BytesType' but was '%s'", 
testTable.DefaultValidator)
-               }
-               expectedKeyAliases := []string{"first_id"}
-               if !reflect.DeepEqual(testTable.KeyAliases, expectedKeyAliases) 
{
-                       t.Errorf("Expected key aliases %v but was %v", 
expectedKeyAliases, testTable.KeyAliases)
-               }
-               expectedColumnAliases := []string{"second_id"}
-               if !reflect.DeepEqual(testTable.ColumnAliases, 
expectedColumnAliases) {
-                       t.Errorf("Expected key aliases %v but was %v", 
expectedColumnAliases, testTable.ColumnAliases)
-               }
-       }
+
        if testTable.ValueAlias != "" {
                t.Errorf("Expected value alias '' but was '%s'", 
testTable.ValueAlias)
        }
@@ -3228,10 +3210,6 @@ func TestUnmarshallNestedTypes(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("can not have frozen types in cassandra < 2.1.3")
-       }
-
        if err := createTable(session, `CREATE TABLE gocql_test.test_557 (
                    id text PRIMARY KEY,
                    val list<frozen<map<text, text> > >
diff --git a/conn.go b/conn.go
index 93f6cab1..b7a3ed27 100644
--- a/conn.go
+++ b/conn.go
@@ -179,7 +179,7 @@ type Conn struct {
        frameObserver  FrameHeaderObserver
        streamObserver StreamObserver
 
-       headerBuf [maxFrameHeaderSize]byte
+       headerBuf [frameHeadSize]byte
 
        streams *streams.IDGenerator
        mu      sync.Mutex
@@ -784,12 +784,11 @@ func (c *Conn) recvSegment(ctx context.Context) error {
                return err
        }
 
-       const frameHeaderLength = 9
-       buf := bytes.NewBuffer(make([]byte, 0, head.length+frameHeaderLength))
+       buf := bytes.NewBuffer(make([]byte, 0, head.length+frameHeadSize))
        buf.Write(frame)
 
        // Computing how many bytes of message left to read
-       bytesToRead := head.length - len(frame) + frameHeaderLength
+       bytesToRead := head.length - len(frame) + frameHeadSize
 
        err = c.recvPartialFrames(buf, bytesToRead)
        if err != nil {
@@ -1738,10 +1737,6 @@ func (c *Conn) UseKeyspace(keyspace string) error {
 }
 
 func (c *Conn) executeBatch(ctx context.Context, batch *Batch) *Iter {
-       if c.version == protoVersion1 {
-               return &Iter{err: ErrUnsupported}
-       }
-
        n := len(batch.Entries)
        req := &writeBatchFrame{
                typ:                   batch.Type,
diff --git a/conn_test.go b/conn_test.go
index f78361d3..cb76d90c 100644
--- a/conn_test.go
+++ b/conn_test.go
@@ -53,7 +53,7 @@ import (
 )
 
 const (
-       defaultProto = protoVersion2
+       defaultProto = protoVersion4
 )
 
 func TestApprove(t *testing.T) {
@@ -1056,18 +1056,13 @@ func (nts newTestServerOpts) newServer(t testing.TB, 
ctx context.Context) *TestS
                t.Fatal(err)
        }
 
-       headerSize := 8
-       if nts.protocol > protoVersion2 {
-               headerSize = 9
-       }
-
        ctx, cancel := context.WithCancel(ctx)
        srv := &TestServer{
                Address:    listen.Addr().String(),
                listen:     listen,
                t:          t,
                protocol:   nts.protocol,
-               headerSize: headerSize,
+               headerSize: 9,
                ctx:        ctx,
                cancel:     cancel,
 
@@ -1103,18 +1098,13 @@ func NewSSLTestServer(t testing.TB, protocol uint8, ctx 
context.Context) *TestSe
                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,
+               headerSize: 9,
                ctx:        ctx,
                cancel:     cancel,
        }
diff --git a/frame.go b/frame.go
index c3b9606b..f2f6c7cd 100644
--- a/frame.go
+++ b/frame.go
@@ -307,7 +307,7 @@ var (
        ErrFrameTooBig = errors.New("frame length is bigger than the maximum 
allowed")
 )
 
-const maxFrameHeaderSize = 9
+const frameHeadSize = 9
 
 func readInt(p []byte) int32 {
        return int32(p[0])<<24 | int32(p[1])<<16 | int32(p[2])<<8 | int32(p[3])
@@ -364,9 +364,8 @@ type FrameHeaderObserver interface {
 type framer struct {
        proto byte
        // flags are for outgoing flags, enabling compression and tracing etc
-       flags    byte
-       compres  Compressor
-       headSize int
+       flags   byte
+       compres Compressor
        // if this frame was read then the header will be here
        header *frameHeader
 
@@ -398,15 +397,9 @@ func newFramer(compressor Compressor, version byte) 
*framer {
 
        version &= protoVersionMask
 
-       headSize := 8
-       if version > protoVersion2 {
-               headSize = 9
-       }
-
        f.compres = compressor
        f.proto = version
        f.flags = flags
-       f.headSize = headSize
 
        f.header = nil
        f.traceID = nil
@@ -426,43 +419,27 @@ func readHeader(r io.Reader, p []byte) (head frameHeader, 
err error) {
 
        version := p[0] & protoVersionMask
 
-       if version < protoVersion1 || version > protoVersion5 {
+       if version < protoVersion3 || version > protoVersion5 {
                return frameHeader{}, fmt.Errorf("gocql: unsupported protocol 
response version: %d", version)
        }
 
-       headSize := 9
-       if version < protoVersion3 {
-               headSize = 8
-       }
-
-       _, err = io.ReadFull(r, p[1:headSize])
+       _, err = io.ReadFull(r, p[1:frameHeadSize])
        if err != nil {
                return frameHeader{}, err
        }
 
-       p = p[:headSize]
+       p = p[:frameHeadSize]
 
        head.version = protoVersion(p[0])
        head.flags = p[1]
 
-       if version > protoVersion2 {
-               if len(p) != 9 {
-                       return frameHeader{}, fmt.Errorf("not enough bytes to 
read header require 9 got: %d", len(p))
-               }
-
-               head.stream = int(int16(p[2])<<8 | int16(p[3]))
-               head.op = frameOp(p[4])
-               head.length = int(readInt(p[5:]))
-       } else {
-               if len(p) != 8 {
-                       return frameHeader{}, fmt.Errorf("not enough bytes to 
read header require 8 got: %d", len(p))
-               }
-
-               head.stream = int(int8(p[2]))
-               head.op = frameOp(p[3])
-               head.length = int(readInt(p[4:]))
+       if len(p) != 9 {
+               return frameHeader{}, fmt.Errorf("not enough bytes to read 
header require 9 got: %d", len(p))
        }
 
+       head.stream = int(int16(p[2])<<8 | int16(p[3]))
+       head.op = frameOp(p[4])
+       head.length = int(readInt(p[5:]))
        return head, nil
 }
 
@@ -700,31 +677,18 @@ func (f *framer) readErrorMap() (errMap ErrorMap) {
 }
 
 func (f *framer) writeHeader(flags byte, op frameOp, stream int) {
-       if f.proto <= protoVersion2 {
-               f.buf = append(f.buf[:0],
-                       f.proto, flags, byte(stream),
-                       // pad out length
-                       byte(op), 0, 0, 0, 0,
-               )
-       } else {
-               f.buf = append(f.buf[:0],
-                       f.proto, flags, byte(stream>>8), byte(stream),
-                       // pad out length
-                       byte(op), 0, 0, 0, 0,
-               )
-       }
+       f.buf = append(f.buf[:0],
+               f.proto, flags, byte(stream>>8), byte(stream),
+               // pad out length
+               byte(op), 0, 0, 0, 0,
+       )
 }
 
 func (f *framer) setLength(length int) {
-       p := 4
-       if f.proto > protoVersion2 {
-               p = 5
-       }
-
-       f.buf[p+0] = byte(length >> 24)
-       f.buf[p+1] = byte(length >> 16)
-       f.buf[p+2] = byte(length >> 8)
-       f.buf[p+3] = byte(length)
+       f.buf[5] = byte(length >> 24)
+       f.buf[6] = byte(length >> 16)
+       f.buf[7] = byte(length >> 8)
+       f.buf[8] = byte(length)
 }
 
 func (f *framer) finish() error {
@@ -740,14 +704,14 @@ func (f *framer) finish() error {
                }
 
                // TODO: only compress frames which are big enough
-               compressed, err := f.compres.AppendCompressedWithLength(nil, 
f.buf[f.headSize:])
+               compressed, err := f.compres.AppendCompressedWithLength(nil, 
f.buf[frameHeadSize:])
                if err != nil {
                        return err
                }
 
-               f.buf = append(f.buf[:f.headSize], compressed...)
+               f.buf = append(f.buf[:frameHeadSize], compressed...)
        }
-       length := len(f.buf) - f.headSize
+       length := len(f.buf) - frameHeadSize
        f.setLength(length)
 
        return nil
@@ -1180,11 +1144,6 @@ func (f *framer) parseResultPrepared() frame {
        }
 
        frame.reqMeta = f.parsePreparedMetadata()
-
-       if f.proto < protoVersion2 {
-               return frame
-       }
-
        frame.respMeta = f.parseResultMetadata()
 
        return frame
@@ -1240,87 +1199,65 @@ type schemaChangeAggregate struct {
 }
 
 func (f *framer) parseResultSchemaChange() frame {
-       if f.proto <= protoVersion2 {
-               change := f.readString()
-               keyspace := f.readString()
-               table := f.readString()
-
-               if table != "" {
-                       return &schemaChangeTable{
-                               frameHeader: *f.header,
-                               change:      change,
-                               keyspace:    keyspace,
-                               object:      table,
-                       }
-               } else {
-                       return &schemaChangeKeyspace{
-                               frameHeader: *f.header,
-                               change:      change,
-                               keyspace:    keyspace,
-                       }
+       change := f.readString()
+       target := f.readString()
+
+       // TODO: could just use a separate type for each target
+       switch target {
+       case "KEYSPACE":
+               frame := &schemaChangeKeyspace{
+                       frameHeader: *f.header,
+                       change:      change,
                }
-       } else {
-               change := f.readString()
-               target := f.readString()
-
-               // TODO: could just use a separate type for each target
-               switch target {
-               case "KEYSPACE":
-                       frame := &schemaChangeKeyspace{
-                               frameHeader: *f.header,
-                               change:      change,
-                       }
 
-                       frame.keyspace = f.readString()
+               frame.keyspace = f.readString()
 
-                       return frame
-               case "TABLE":
-                       frame := &schemaChangeTable{
-                               frameHeader: *f.header,
-                               change:      change,
-                       }
+               return frame
+       case "TABLE":
+               frame := &schemaChangeTable{
+                       frameHeader: *f.header,
+                       change:      change,
+               }
 
-                       frame.keyspace = f.readString()
-                       frame.object = f.readString()
+               frame.keyspace = f.readString()
+               frame.object = f.readString()
 
-                       return frame
-               case "TYPE":
-                       frame := &schemaChangeType{
-                               frameHeader: *f.header,
-                               change:      change,
-                       }
+               return frame
+       case "TYPE":
+               frame := &schemaChangeType{
+                       frameHeader: *f.header,
+                       change:      change,
+               }
 
-                       frame.keyspace = f.readString()
-                       frame.object = f.readString()
+               frame.keyspace = f.readString()
+               frame.object = f.readString()
 
-                       return frame
-               case "FUNCTION":
-                       frame := &schemaChangeFunction{
-                               frameHeader: *f.header,
-                               change:      change,
-                       }
+               return frame
+       case "FUNCTION":
+               frame := &schemaChangeFunction{
+                       frameHeader: *f.header,
+                       change:      change,
+               }
 
-                       frame.keyspace = f.readString()
-                       frame.name = f.readString()
-                       frame.args = f.readStringList()
+               frame.keyspace = f.readString()
+               frame.name = f.readString()
+               frame.args = f.readStringList()
 
-                       return frame
-               case "AGGREGATE":
-                       frame := &schemaChangeAggregate{
-                               frameHeader: *f.header,
-                               change:      change,
-                       }
+               return frame
+       case "AGGREGATE":
+               frame := &schemaChangeAggregate{
+                       frameHeader: *f.header,
+                       change:      change,
+               }
 
-                       frame.keyspace = f.readString()
-                       frame.name = f.readString()
-                       frame.args = f.readStringList()
+               frame.keyspace = f.readString()
+               frame.name = f.readString()
+               frame.args = f.readStringList()
 
-                       return frame
-               default:
-                       panic(fmt.Errorf("gocql: unknown SCHEMA_CHANGE target: 
%q change: %q", target, change))
-               }
+               return frame
+       default:
+               panic(fmt.Errorf("gocql: unknown SCHEMA_CHANGE target: %q 
change: %q", target, change))
        }
-
 }
 
 type authenticateFrame struct {
@@ -1474,10 +1411,6 @@ func (q queryParams) String() string {
 func (f *framer) writeQueryParams(opts *queryParams) {
        f.writeConsistency(opts.consistency)
 
-       if f.proto == protoVersion1 {
-               return
-       }
-
        var flags uint32
        names := false
 
@@ -1497,16 +1430,13 @@ func (f *framer) writeQueryParams(opts *queryParams) {
                flags |= flagWithSerialConsistency
        }
 
-       // protoV3 specific things
-       if f.proto > protoVersion2 {
-               if opts.defaultTimestamp {
-                       flags |= flagDefaultTimestamp
-               }
+       if opts.defaultTimestamp {
+               flags |= flagDefaultTimestamp
+       }
 
-               if len(opts.values) > 0 && opts.values[0].name != "" {
-                       flags |= flagWithNameValues
-                       names = true
-               }
+       if len(opts.values) > 0 && opts.values[0].name != "" {
+               flags |= flagWithNameValues
+               names = true
        }
 
        if opts.keyspace != "" {
@@ -1556,7 +1486,7 @@ func (f *framer) writeQueryParams(opts *queryParams) {
                f.writeConsistency(opts.serialConsistency)
        }
 
-       if f.proto > protoVersion2 && opts.defaultTimestamp {
+       if opts.defaultTimestamp {
                // timestamp in microseconds
                var ts int64
                if opts.defaultTimestampValue != 0 {
@@ -1645,20 +1575,7 @@ func (f *framer) writeExecuteFrame(streamID int, 
preparedID, resultMetadataID []
                f.writeShortBytes(resultMetadataID)
        }
 
-       if f.proto > protoVersion1 {
-               f.writeQueryParams(params)
-       } else {
-               n := len(params.values)
-               f.writeShort(uint16(n))
-               for i := 0; i < n; i++ {
-                       if params.values[i].isUnset {
-                               f.writeUnset()
-                       } else {
-                               f.writeBytes(params.values[i].value)
-                       }
-               }
-               f.writeConsistency(params.consistency)
-       }
+       f.writeQueryParams(params)
 
        return f.finish()
 }
@@ -1719,7 +1636,7 @@ func (f *framer) writeBatchFrame(streamID int, w 
*writeBatchFrame, customPayload
                f.writeShort(uint16(len(b.values)))
                for j := range b.values {
                        col := b.values[j]
-                       if f.proto > protoVersion2 && col.name != "" {
+                       if col.name != "" {
                                // TODO: move this check into the caller and 
set a flag on writeBatchFrame
                                // to indicate using named values
                                if f.proto <= protoVersion5 {
@@ -1738,13 +1655,12 @@ func (f *framer) writeBatchFrame(streamID int, w 
*writeBatchFrame, customPayload
 
        f.writeConsistency(w.consistency)
 
-       if f.proto > protoVersion2 {
-               if w.serialConsistency > 0 {
-                       flags |= flagWithSerialConsistency
-               }
-               if w.defaultTimestamp {
-                       flags |= flagDefaultTimestamp
-               }
+       if w.serialConsistency > 0 {
+               flags |= flagWithSerialConsistency
+       }
+
+       if w.defaultTimestamp {
+               flags |= flagDefaultTimestamp
        }
 
        if w.keyspace != "" {
diff --git a/frame_test.go b/frame_test.go
index 8cb9024a..6bf49222 100644
--- a/frame_test.go
+++ b/frame_test.go
@@ -108,14 +108,14 @@ func TestFrameReadTooLong(t *testing.T) {
        r := &bytes.Buffer{}
        r.Write(make([]byte, maxFrameSize+1))
        // write a new header right after this frame to verify that we can read 
it
-       r.Write([]byte{0x02, 0x00, 0x00, byte(opReady), 0x00, 0x00, 0x00, 0x00})
+       r.Write([]byte{protoVersionMask & protoVersion3, 0x00, 0x00, 0x00, 
byte(opReady), 0x00, 0x00, 0x00, 0x00})
 
-       framer := newFramer(nil, 2)
+       framer := newFramer(nil, 3)
 
        head := frameHeader{
-               version: 2,
+               version: protoVersion3,
                op:      opReady,
-               length:  r.Len() - 8,
+               length:  r.Len() - frameHeadSize,
        }
 
        err := framer.readFrame(r, &head)
@@ -123,7 +123,7 @@ func TestFrameReadTooLong(t *testing.T) {
                t.Fatalf("expected to get %v got %v", ErrFrameTooBig, err)
        }
 
-       head, err = readHeader(r, make([]byte, 8))
+       head, err = readHeader(r, make([]byte, frameHeadSize))
        if err != nil {
                t.Fatal(err)
        }
diff --git a/marshal.go b/marshal.go
index d0cf6730..65ab656c 100644
--- a/marshal.go
+++ b/marshal.go
@@ -116,7 +116,6 @@ func Marshal(info TypeInfo, value interface{}) ([]byte, 
error) {
        if info.Version() < protoVersion1 {
                panic("protocol version not set")
        }
-
        if valueRef := reflect.ValueOf(value); valueRef.Kind() == reflect.Ptr {
                if valueRef.IsNil() {
                        return nil, nil
@@ -1565,25 +1564,15 @@ func encVint(v int64) []byte {
        return buf
 }
 
-func writeCollectionSize(info CollectionType, n int, buf *bytes.Buffer) error {
-       if info.proto > protoVersion2 {
-               if n > math.MaxInt32 {
-                       return marshalErrorf("marshal: collection too large")
-               }
-
-               buf.WriteByte(byte(n >> 24))
-               buf.WriteByte(byte(n >> 16))
-               buf.WriteByte(byte(n >> 8))
-               buf.WriteByte(byte(n))
-       } else {
-               if n > math.MaxUint16 {
-                       return marshalErrorf("marshal: collection too large")
-               }
-
-               buf.WriteByte(byte(n >> 8))
-               buf.WriteByte(byte(n))
+func writeCollectionSize(n int, buf *bytes.Buffer) error {
+       if n > math.MaxInt32 {
+               return marshalErrorf("marshal: collection too large")
        }
 
+       buf.WriteByte(byte(n >> 24))
+       buf.WriteByte(byte(n >> 16))
+       buf.WriteByte(byte(n >> 8))
+       buf.WriteByte(byte(n))
        return nil
 }
 
@@ -1611,7 +1600,7 @@ func marshalList(info TypeInfo, value interface{}) 
([]byte, error) {
                buf := &bytes.Buffer{}
                n := rv.Len()
 
-               if err := writeCollectionSize(listInfo, n, buf); err != nil {
+               if err := writeCollectionSize(n, buf); err != nil {
                        return nil, err
                }
 
@@ -1622,10 +1611,10 @@ func marshalList(info TypeInfo, value interface{}) 
([]byte, error) {
                        }
                        itemLen := len(item)
                        // Set the value to null for supported protocols
-                       if item == nil && listInfo.proto > protoVersion2 {
+                       if item == nil {
                                itemLen = -1
                        }
-                       if err := writeCollectionSize(listInfo, itemLen, buf); 
err != nil {
+                       if err := writeCollectionSize(itemLen, buf); err != nil 
{
                                return nil, err
                        }
                        buf.Write(item)
@@ -1645,20 +1634,12 @@ func marshalList(info TypeInfo, value interface{}) 
([]byte, error) {
        return nil, marshalErrorf("can not marshal %T into %s. Accepted types: 
slice, array, map[]struct.", value, info)
 }
 
-func readCollectionSize(info CollectionType, data []byte) (size, read int, err 
error) {
-       if info.proto > protoVersion2 {
-               if len(data) < 4 {
-                       return 0, 0, unmarshalErrorf("unmarshal list: 
unexpected eof")
-               }
-               size = int(int32(data[0])<<24 | int32(data[1])<<16 | 
int32(data[2])<<8 | int32(data[3]))
-               read = 4
-       } else {
-               if len(data) < 2 {
-                       return 0, 0, unmarshalErrorf("unmarshal list: 
unexpected eof")
-               }
-               size = int(data[0])<<8 | int(data[1])
-               read = 2
+func readCollectionSize(data []byte) (size, read int, err error) {
+       if len(data) < 4 {
+               return 0, 0, unmarshalErrorf("unmarshal list: unexpected eof")
        }
+       size = int(int32(data[0])<<24 | int32(data[1])<<16 | int32(data[2])<<8 
| int32(data[3]))
+       read = 4
        return
 }
 
@@ -1688,7 +1669,7 @@ func unmarshalList(info TypeInfo, data []byte, value 
interface{}) error {
                        rv.Set(reflect.Zero(t))
                        return nil
                }
-               n, p, err := readCollectionSize(listInfo, data)
+               n, p, err := readCollectionSize(data)
                if err != nil {
                        return err
                }
@@ -1701,7 +1682,7 @@ func unmarshalList(info TypeInfo, data []byte, value 
interface{}) error {
                        rv.Set(reflect.MakeSlice(t, n, n))
                }
                for i := 0; i < n; i++ {
-                       m, p, err := readCollectionSize(listInfo, data)
+                       m, p, err := readCollectionSize(data)
                        if err != nil {
                                return err
                        }
@@ -1909,7 +1890,7 @@ func marshalMap(info TypeInfo, value interface{}) 
([]byte, error) {
        buf := &bytes.Buffer{}
        n := rv.Len()
 
-       if err := writeCollectionSize(mapInfo, n, buf); err != nil {
+       if err := writeCollectionSize(n, buf); err != nil {
                return nil, err
        }
 
@@ -1921,10 +1902,10 @@ func marshalMap(info TypeInfo, value interface{}) 
([]byte, error) {
                }
                itemLen := len(item)
                // Set the key to null for supported protocols
-               if item == nil && mapInfo.proto > protoVersion2 {
+               if item == nil {
                        itemLen = -1
                }
-               if err := writeCollectionSize(mapInfo, itemLen, buf); err != 
nil {
+               if err := writeCollectionSize(itemLen, buf); err != nil {
                        return nil, err
                }
                buf.Write(item)
@@ -1935,10 +1916,10 @@ func marshalMap(info TypeInfo, value interface{}) 
([]byte, error) {
                }
                itemLen = len(item)
                // Set the value to null for supported protocols
-               if item == nil && mapInfo.proto > protoVersion2 {
+               if item == nil {
                        itemLen = -1
                }
-               if err := writeCollectionSize(mapInfo, itemLen, buf); err != 
nil {
+               if err := writeCollectionSize(itemLen, buf); err != nil {
                        return nil, err
                }
                buf.Write(item)
@@ -1965,7 +1946,7 @@ func unmarshalMap(info TypeInfo, data []byte, value 
interface{}) error {
                rv.Set(reflect.Zero(t))
                return nil
        }
-       n, p, err := readCollectionSize(mapInfo, data)
+       n, p, err := readCollectionSize(data)
        if err != nil {
                return err
        }
@@ -1975,7 +1956,7 @@ func unmarshalMap(info TypeInfo, data []byte, value 
interface{}) error {
        rv.Set(reflect.MakeMapWithSize(t, n))
        data = data[p:]
        for i := 0; i < n; i++ {
-               m, p, err := readCollectionSize(mapInfo, data)
+               m, p, err := readCollectionSize(data)
                if err != nil {
                        return err
                }
@@ -1994,7 +1975,7 @@ func unmarshalMap(info TypeInfo, data []byte, value 
interface{}) error {
                        return err
                }
 
-               m, p, err = readCollectionSize(mapInfo, data)
+               m, p, err = readCollectionSize(data)
                if err != nil {
                        return err
                }
diff --git a/marshal_test.go b/marshal_test.go
index 969d8e5f..cdbc6467 100644
--- a/marshal_test.go
+++ b/marshal_test.go
@@ -59,56 +59,56 @@ var marshalTests = []struct {
        UnmarshalError error
 }{
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte("hello world"),
                []byte("hello world"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte("hello world"),
                "hello world",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte(nil),
                []byte(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte("hello world"),
                MyString("hello world"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte("HELLO WORLD"),
                CustomString("hello world"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBlob},
+               NativeType{proto: protoVersion3, typ: TypeBlob},
                []byte("hello\x00"),
                []byte("hello\x00"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBlob},
+               NativeType{proto: protoVersion3, typ: TypeBlob},
                []byte(nil),
                []byte(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimeUUID},
+               NativeType{proto: protoVersion3, typ: TypeTimeUUID},
                []byte{0x3d, 0xcd, 0x98, 0x0, 0xf3, 0xd9, 0x11, 0xbf, 0x86, 
0xd4, 0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                func() UUID {
                        x, _ := UUIDFromBytes([]byte{0x3d, 0xcd, 0x98, 0x0, 
0xf3, 0xd9, 0x11, 0xbf, 0x86, 0xd4, 0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0})
@@ -118,287 +118,287 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimeUUID},
+               NativeType{proto: protoVersion3, typ: TypeTimeUUID},
                []byte{0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                []byte{0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                MarshalError("can not marshal []byte 6 bytes long into 
timeuuid, must be exactly 16 bytes long"),
                UnmarshalError("unable to parse UUID: UUIDs must be exactly 16 
bytes long"),
        },
        {
-               NativeType{proto: 2, typ: TypeTimeUUID},
+               NativeType{proto: protoVersion3, typ: TypeTimeUUID},
                []byte{0x3d, 0xcd, 0x98, 0x0, 0xf3, 0xd9, 0x11, 0xbf, 0x86, 
0xd4, 0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                [16]byte{0x3d, 0xcd, 0x98, 0x0, 0xf3, 0xd9, 0x11, 0xbf, 0x86, 
0xd4, 0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x00\x00\x00\x00"),
                0,
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x01\x02\x03\x04"),
                int(16909060),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x01\x02\x03\x04"),
                AliasInt(16909060),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x80\x00\x00\x00"),
                int32(math.MinInt32),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x7f\xff\xff\xff"),
                int32(math.MaxInt32),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x00\x00\x00\x00"),
                "0",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x01\x02\x03\x04"),
                "16909060",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x80\x00\x00\x00"),
                "-2147483648", // math.MinInt32
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x7f\xff\xff\xff"),
                "2147483647", // math.MaxInt32
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x00\x00"),
                0,
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x01\x02\x03\x04\x05\x06\x07\x08"),
                72623859790382856,
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x80\x00\x00\x00\x00\x00\x00\x00"),
                int64(math.MinInt64),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x7f\xff\xff\xff\xff\xff\xff\xff"),
                int64(math.MaxInt64),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x00\x00"),
                "0",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x01\x02\x03\x04\x05\x06\x07\x08"),
                "72623859790382856",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x80\x00\x00\x00\x00\x00\x00\x00"),
                "-9223372036854775808", // math.MinInt64
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x7f\xff\xff\xff\xff\xff\xff\xff"),
                "9223372036854775807", // math.MaxInt64
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBoolean},
+               NativeType{proto: protoVersion3, typ: TypeBoolean},
                []byte("\x00"),
                false,
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBoolean},
+               NativeType{proto: protoVersion3, typ: TypeBoolean},
                []byte("\x01"),
                true,
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeFloat},
+               NativeType{proto: protoVersion3, typ: TypeFloat},
                []byte("\x40\x49\x0f\xdb"),
                float32(3.14159265),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDouble},
+               NativeType{proto: protoVersion3, typ: TypeDouble},
                []byte("\x40\x09\x21\xfb\x53\xc8\xd4\xf1"),
                float64(3.14159265),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x00\x00"),
                inf.NewDec(0, 0),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x00\x64"),
                inf.NewDec(100, 0),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x02\x19"),
                decimalize("0.25"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x13\xD5\a;\x20\x14\xA2\x91"),
                decimalize("-0.0012095473475870063"), // From the 
iconara/cql-rb test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x13*\xF8\xC4\xDF\xEB]o"),
                decimalize("0.0012095473475870063"), // From the iconara/cql-rb 
test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                
[]byte("\x00\x00\x00\x12\xF2\xD8\x02\xB6R\x7F\x99\xEE\x98#\x99\xA9V"),
                decimalize("-1042342234234.123423435647768234"), // From the 
iconara/cql-rb test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\r\nJ\x04\"^\x91\x04\x8a\xb1\x18\xfe"),
                decimalize("1243878957943.1234124191998"), // From the 
datastax/python-driver test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x06\xe5\xde]\x98Y"),
                decimalize("-112233.441191"), // From the 
datastax/python-driver test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x14\x00\xfa\xce"),
                decimalize("0.00000000000000064206"), // From the 
datastax/python-driver test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\x00\x00\x00\x14\xff\x052"),
                decimalize("-0.00000000000000064206"), // From the 
datastax/python-driver test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\xff\xff\xff\x9c\x00\xfa\xce"),
                inf.NewDec(64206, -100), // From the datastax/python-driver 
test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 4, typ: TypeTime},
+               NativeType{proto: protoVersion4, typ: TypeTime},
                []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                time.Duration(int64(1376387523000)),
                nil,
                nil,
        },
        {
-               NativeType{proto: 4, typ: TypeTime},
+               NativeType{proto: protoVersion4, typ: TypeTime},
                []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                int64(1376387523000),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimestamp},
+               NativeType{proto: protoVersion3, typ: TypeTimestamp},
                []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                time.Date(2013, time.August, 13, 9, 52, 3, 0, time.UTC),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimestamp},
+               NativeType{proto: protoVersion3, typ: TypeTimestamp},
                []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                int64(1376387523000),
                nil,
                nil,
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89\xa2\xc3\xc2\x9a\xe0F\x91\x06"),
                Duration{Months: 1233, Days: 123213, Nanoseconds: 2312323},
                nil,
                nil,
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89\xa1\xc3\xc2\x99\xe0F\x91\x05"),
                Duration{Months: -1233, Days: -123213, Nanoseconds: -2312323},
                nil,
                nil,
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x02\x04\x80\xe6"),
                Duration{Months: 1, Days: 2, Nanoseconds: 115},
                nil,
@@ -406,73 +406,73 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeList},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               
[]byte("\x00\x02\x00\x04\x00\x00\x00\x01\x00\x04\x00\x00\x00\x02"),
+               
[]byte("\x00\x00\x00\x02\x00\x00\x00\x04\x00\x00\x00\x01\x00\x00\x00\x04\x00\x00\x00\x02"),
                []int{1, 2},
                nil,
                nil,
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeList},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               
[]byte("\x00\x02\x00\x04\x00\x00\x00\x01\x00\x04\x00\x00\x00\x02"),
+               
[]byte("\x00\x00\x00\x02\x00\x00\x00\x04\x00\x00\x00\x01\x00\x00\x00\x04\x00\x00\x00\x02"),
                [2]int{1, 2},
                nil,
                nil,
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeSet},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeSet},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               
[]byte("\x00\x02\x00\x04\x00\x00\x00\x01\x00\x04\x00\x00\x00\x02"),
+               
[]byte("\x00\x00\x00\x02\x00\x00\x00\x04\x00\x00\x00\x01\x00\x00\x00\x04\x00\x00\x00\x02"),
                []int{1, 2},
                nil,
                nil,
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeSet},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeSet},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               []byte{0, 0}, // encoding of a list should always include the 
size of the collection
+               []byte{0, 0, 0, 0}, // encoding of a list should always include 
the size of the collection
                []int{},
                nil,
                nil,
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               []byte("\x00\x01\x00\x03foo\x00\x04\x00\x00\x00\x01"),
+               
[]byte("\x00\x00\x00\x01\x00\x00\x00\x03foo\x00\x00\x00\x04\x00\x00\x00\x01"),
                map[string]int{"foo": 1},
                nil,
                nil,
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               []byte{0, 0},
+               []byte{0, 0, 0, 0},
                map[string]int{},
                nil,
                nil,
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeList},
-                       Elem:       NativeType{proto: 2, typ: TypeVarchar},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeVarchar},
                },
                bytes.Join([][]byte{
-                       []byte("\x00\x01\xFF\xFF"),
+                       []byte("\x00\x00\x00\x01\x00\x00\xff\xff"),
                        bytes.Repeat([]byte("X"), math.MaxUint16)}, []byte("")),
                []string{strings.Repeat("X", math.MaxUint16)},
                nil,
@@ -480,14 +480,14 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeVarchar},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeVarchar},
                },
                bytes.Join([][]byte{
-                       []byte("\x00\x01\xFF\xFF"),
+                       []byte("\x00\x00\x00\x01\x00\x00\xff\xff"),
                        bytes.Repeat([]byte("X"), math.MaxUint16),
-                       []byte("\xFF\xFF"),
+                       []byte("\x00\x00\xFF\xFF"),
                        bytes.Repeat([]byte("Y"), math.MaxUint16)}, []byte("")),
                map[string]string{
                        strings.Repeat("X", math.MaxUint16): 
strings.Repeat("Y", math.MaxUint16),
@@ -496,119 +496,119 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte("\x00"),
                0,
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte("\x37\xE2\x3C\xEC"),
                int32(937573612),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte("\x37\xE2\x3C\xEC"),
                big.NewInt(937573612),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte("\x03\x9EV \x15\f\x03\x9DK\x18\xCDI\\$?\a["),
                bigintize("1231312312331283012830129382342342412123"), // From 
the iconara/cql-rb test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte("\xC9v\x8D:\x86"),
                big.NewInt(-234234234234), // From the iconara/cql-rb test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte("f\x1e\xfd\xf2\xe3\xb1\x9f|\x04_\x15"),
                bigintize("123456789123456789123456789"), // From the 
datastax/python-driver test suite
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarint},
+               NativeType{proto: protoVersion3, typ: TypeVarint},
                []byte(nil),
                nil,
                nil,
                UnmarshalError("can not unmarshal into non-pointer <nil>"),
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                []byte("\x7F\x00\x00\x01"),
                net.ParseIP("127.0.0.1").To4(),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                []byte("\xFF\xFF\xFF\xFF"),
                net.ParseIP("255.255.255.255").To4(),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                []byte("\x7F\x00\x00\x01"),
                "127.0.0.1",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                []byte("\xFF\xFF\xFF\xFF"),
                "255.255.255.255",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                
[]byte("\x21\xDA\x00\xd3\x00\x00\x2f\x3b\x02\xaa\x00\xff\xfe\x28\x9c\x5a"),
                "21da:d3:0:2f3b:2aa:ff:fe28:9c5a",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                
[]byte("\xfe\x80\x00\x00\x00\x00\x00\x00\x02\x02\xb3\xff\xfe\x1e\x83\x29"),
                "fe80::202:b3ff:fe1e:8329",
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                
[]byte("\x21\xDA\x00\xd3\x00\x00\x2f\x3b\x02\xaa\x00\xff\xfe\x28\x9c\x5a"),
                net.ParseIP("21da:d3:0:2f3b:2aa:ff:fe28:9c5a"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                
[]byte("\xfe\x80\x00\x00\x00\x00\x00\x00\x02\x02\xb3\xff\xfe\x1e\x83\x29"),
                net.ParseIP("fe80::202:b3ff:fe1e:8329"),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte(nil),
                nil,
                nil,
                UnmarshalError("can not unmarshal into non-pointer <nil>"),
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte("nullable string"),
                func() *string {
                        value := "nullable string"
@@ -618,14 +618,14 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte(nil),
                (*string)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x7f\xff\xff\xff"),
                func() *int {
                        var value int = math.MaxInt32
@@ -635,28 +635,28 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte(nil),
                (*int)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimeUUID},
+               NativeType{proto: protoVersion3, typ: TypeTimeUUID},
                []byte{0x3d, 0xcd, 0x98, 0x0, 0xf3, 0xd9, 0x11, 0xbf, 0x86, 
0xd4, 0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                &UUID{0x3d, 0xcd, 0x98, 0x0, 0xf3, 0xd9, 0x11, 0xbf, 0x86, 
0xd4, 0xb8, 0xe8, 0x56, 0x2c, 0xc, 0xd0},
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimeUUID},
+               NativeType{proto: protoVersion3, typ: TypeTimeUUID},
                []byte(nil),
                (*UUID)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimestamp},
+               NativeType{proto: protoVersion3, typ: TypeTimestamp},
                []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                func() *time.Time {
                        t := time.Date(2013, time.August, 13, 9, 52, 3, 0, 
time.UTC)
@@ -666,14 +666,14 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTimestamp},
+               NativeType{proto: protoVersion3, typ: TypeTimestamp},
                []byte(nil),
                (*time.Time)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBoolean},
+               NativeType{proto: protoVersion3, typ: TypeBoolean},
                []byte("\x00"),
                func() *bool {
                        b := false
@@ -683,7 +683,7 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBoolean},
+               NativeType{proto: protoVersion3, typ: TypeBoolean},
                []byte("\x01"),
                func() *bool {
                        b := true
@@ -693,14 +693,14 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBoolean},
+               NativeType{proto: protoVersion3, typ: TypeBoolean},
                []byte(nil),
                (*bool)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeFloat},
+               NativeType{proto: protoVersion3, typ: TypeFloat},
                []byte("\x40\x49\x0f\xdb"),
                func() *float32 {
                        f := float32(3.14159265)
@@ -710,14 +710,14 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeFloat},
+               NativeType{proto: protoVersion3, typ: TypeFloat},
                []byte(nil),
                (*float32)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDouble},
+               NativeType{proto: protoVersion3, typ: TypeDouble},
                []byte("\x40\x09\x21\xfb\x53\xc8\xd4\xf1"),
                func() *float64 {
                        d := float64(3.14159265)
@@ -727,14 +727,14 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeDouble},
+               NativeType{proto: protoVersion3, typ: TypeDouble},
                []byte(nil),
                (*float64)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                []byte("\x7F\x00\x00\x01"),
                func() *net.IP {
                        ip := net.ParseIP("127.0.0.1").To4()
@@ -744,7 +744,7 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInet},
+               NativeType{proto: protoVersion3, typ: TypeInet},
                []byte(nil),
                (*net.IP)(nil),
                nil,
@@ -752,10 +752,10 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeList},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               
[]byte("\x00\x02\x00\x04\x00\x00\x00\x01\x00\x04\x00\x00\x00\x02"),
+               
[]byte("\x00\x00\x00\x02\x00\x00\x00\x04\x00\x00\x00\x01\x00\x00\x00\x04\x00\x00\x00\x02"),
                func() *[]int {
                        l := []int{1, 2}
                        return &l
@@ -765,8 +765,8 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 3, typ: TypeList},
-                       Elem:       NativeType{proto: 3, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
                
[]byte("\x00\x00\x00\x02\x00\x00\x00\x04\x00\x00\x00\x01\x00\x00\x00\x04\x00\x00\x00\x02"),
                func() *[]int {
@@ -778,8 +778,8 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeList},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
                []byte(nil),
                (*[]int)(nil),
@@ -788,11 +788,11 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               []byte("\x00\x01\x00\x03foo\x00\x04\x00\x00\x00\x01"),
+               
[]byte("\x00\x00\x00\x01\x00\x00\x00\x03foo\x00\x00\x00\x04\x00\x00\x00\x01"),
                func() *map[string]int {
                        m := map[string]int{"foo": 1}
                        return &m
@@ -802,9 +802,9 @@ var marshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
                []byte(nil),
                (*map[string]int)(nil),
@@ -812,7 +812,7 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte("HELLO WORLD"),
                func() *CustomString {
                        customString := CustomString("hello world")
@@ -822,252 +822,252 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte(nil),
                (*CustomString)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x7f\xff"),
                32767, // math.MaxInt16
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x7f\xff"),
                "32767", // math.MaxInt16
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x00\x01"),
                int16(1),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                int16(-1),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x00\xff"),
                uint8(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                uint16(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                uint32(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                uint64(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x00\xff"),
                AliasUint8(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                AliasUint16(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                AliasUint32(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                AliasUint64(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                AliasUint(65535),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\x7f"),
                127, // math.MaxInt8
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\x7f"),
                "127", // math.MaxInt8
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\x01"),
                int16(1),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                int16(-1),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                uint8(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                uint64(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                uint32(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                uint16(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                uint(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                AliasUint8(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                AliasUint64(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                AliasUint32(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                AliasUint16(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTinyInt},
+               NativeType{proto: protoVersion3, typ: TypeTinyInt},
                []byte("\xff"),
                AliasUint(255),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x00\xff"),
                uint8(math.MaxUint8),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\xff\xff"),
                uint64(math.MaxUint16),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\xff\xff\xff\xff"),
                uint64(math.MaxUint32),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                uint64(math.MaxUint64),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\xff\xff\xff\xff"),
                uint32(math.MaxUint32),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\xff\xff\xff\xff"),
                uint64(math.MaxUint32),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeBlob},
+               NativeType{proto: protoVersion3, typ: TypeBlob},
                []byte(nil),
                ([]byte)(nil),
                nil,
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeVarchar},
+               NativeType{proto: protoVersion3, typ: TypeVarchar},
                []byte{},
                func() interface{} {
                        var s string
@@ -1077,7 +1077,7 @@ var marshalTests = []struct {
                nil,
        },
        {
-               NativeType{proto: 2, typ: TypeTime},
+               NativeType{proto: protoVersion3, typ: TypeTime},
                encBigInt(1000),
                time.Duration(1000),
                nil,
@@ -1092,177 +1092,177 @@ var unmarshalTests = []struct {
        UnmarshalError error
 }{
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                uint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x01\x00"),
                uint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\xff\xff\xff\xff"),
                uint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x00\x00\x01\x00"),
                uint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\xff\xff\xff\xff"),
                uint16(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x00\x01\x00\x00"),
                uint16(0),
                UnmarshalError("unmarshal int: value 65536 out of range for 
uint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                uint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x01\x00"),
                uint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                uint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x01\x00"),
                uint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
uint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                uint16(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x01\x00\x00"),
                uint16(0),
                UnmarshalError("unmarshal int: value 65536 out of range for 
uint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                uint32(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
uint32"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x01\x00\x00\x00\x00"),
                uint32(0),
                UnmarshalError("unmarshal int: value 4294967296 out of range 
for uint32"),
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\xff\xff"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeSmallInt},
+               NativeType{proto: protoVersion3, typ: TypeSmallInt},
                []byte("\x01\x00"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\xff\xff\xff\xff"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x00\x00\x01\x00"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\xff\xff\xff\xff"),
                AliasUint16(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeInt},
+               NativeType{proto: protoVersion3, typ: TypeInt},
                []byte("\x00\x01\x00\x00"),
                AliasUint16(0),
                UnmarshalError("unmarshal int: value 65536 out of range for 
gocql.AliasUint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x01\x00"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x00\x01\x00"),
                AliasUint8(0),
                UnmarshalError("unmarshal int: value 256 out of range for 
gocql.AliasUint8"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                AliasUint16(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x00\x00\x01\x00\x00"),
                AliasUint16(0),
                UnmarshalError("unmarshal int: value 65536 out of range for 
gocql.AliasUint16"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\xff\xff\xff\xff\xff\xff\xff\xff"),
                AliasUint32(0),
                UnmarshalError("unmarshal int: value -1 out of range for 
gocql.AliasUint32"),
        },
        {
-               NativeType{proto: 2, typ: TypeBigInt},
+               NativeType{proto: protoVersion3, typ: TypeBigInt},
                []byte("\x00\x00\x00\x01\x00\x00\x00\x00"),
                AliasUint32(0),
                UnmarshalError("unmarshal int: value 4294967296 out of range 
for gocql.AliasUint32"),
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 3, typ: TypeList},
-                       Elem:       NativeType{proto: 3, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeList},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
                []byte("\x00\x00\x00\x02\x00\x00\x00\x04\x00\x00"), // 
truncated data
                func() *[]int {
@@ -1273,56 +1273,56 @@ var unmarshalTests = []struct {
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               []byte("\x00\x01\x00\x03fo"),
+               []byte("\x00\x00\x00\x01\x00\x00\x00\x03fo"),
                map[string]int{"foo": 1},
                UnmarshalError("unmarshal map: unexpected eof"),
        },
        {
                CollectionType{
-                       NativeType: NativeType{proto: 2, typ: TypeMap},
-                       Key:        NativeType{proto: 2, typ: TypeVarchar},
-                       Elem:       NativeType{proto: 2, typ: TypeInt},
+                       NativeType: NativeType{proto: protoVersion3, typ: 
TypeMap},
+                       Key:        NativeType{proto: protoVersion3, typ: 
TypeVarchar},
+                       Elem:       NativeType{proto: protoVersion3, typ: 
TypeInt},
                },
-               []byte("\x00\x01\x00\x03foo\x00\x04\x00\x00"),
+               []byte("\x00\x00\x00\x01\x00\x00\x00\x03foo\x00\x04\x00\x00"),
                map[string]int{"foo": 1},
                UnmarshalError("unmarshal map: unexpected eof"),
        },
        {
-               NativeType{proto: 2, typ: TypeDecimal},
+               NativeType{proto: protoVersion3, typ: TypeDecimal},
                []byte("\xff\xff\xff"),
                inf.NewDec(0, 0), // From the datastax/python-driver test suite
                UnmarshalError("inf.Dec needs at least 4 bytes, while value has 
only 3"),
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89\xa2\xc3\xc2\x9a\xe0F\x91"),
                Duration{},
                UnmarshalError("failed to unmarshal duration into 
*gocql.Duration: failed to extract nanoseconds: data expect to have 9 bytes, 
but it has only 8"),
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89\xa2\xc3\xc2\x9a"),
                Duration{},
                UnmarshalError("failed to unmarshal duration into 
*gocql.Duration: failed to extract nanoseconds: unexpected eof"),
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89\xa2\xc3\xc2"),
                Duration{},
                UnmarshalError("failed to unmarshal duration into 
*gocql.Duration: failed to extract days: data expect to have 5 bytes, but it 
has only 4"),
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89\xa2"),
                Duration{},
                UnmarshalError("failed to unmarshal duration into 
*gocql.Duration: failed to extract days: unexpected eof"),
        },
        {
-               NativeType{proto: 5, typ: TypeDuration},
+               NativeType{proto: protoVersion5, typ: TypeDuration},
                []byte("\x89"),
                Duration{},
                UnmarshalError("failed to unmarshal duration into 
*gocql.Duration: failed to extract month: data expect to have 2 bytes, but it 
has only 1"),
@@ -1459,7 +1459,7 @@ func TestMarshalVarint(t *testing.T) {
        }
 
        for i, test := range varintTests {
-               data, err := Marshal(NativeType{proto: 2, typ: TypeVarint}, 
test.Value)
+               data, err := Marshal(NativeType{proto: protoVersion3, typ: 
TypeVarint}, test.Value)
                if err != nil {
                        t.Errorf("error marshaling varint: %v (test #%d)", err, 
i)
                }
@@ -1469,7 +1469,7 @@ func TestMarshalVarint(t *testing.T) {
                }
 
                binder := new(big.Int)
-               err = Unmarshal(NativeType{proto: 2, typ: TypeVarint}, 
test.Marshaled, binder)
+               err = Unmarshal(NativeType{proto: protoVersion3, typ: 
TypeVarint}, test.Marshaled, binder)
                if err != nil {
                        t.Errorf("error unmarshaling varint: %v (test #%d)", 
err, i)
                }
@@ -1517,7 +1517,7 @@ func TestMarshalVarint(t *testing.T) {
        }
 
        for i, test := range varintUint64Tests {
-               data, err := Marshal(NativeType{proto: 2, typ: TypeVarint}, 
test.Value)
+               data, err := Marshal(NativeType{proto: protoVersion3, typ: 
TypeVarint}, test.Value)
                if err != nil {
                        t.Errorf("error marshaling varint: %v (test #%d)", err, 
i)
                }
@@ -1527,7 +1527,7 @@ func TestMarshalVarint(t *testing.T) {
                }
 
                var binder uint64
-               err = Unmarshal(NativeType{proto: 2, typ: TypeVarint}, 
test.Marshaled, &binder)
+               err = Unmarshal(NativeType{proto: protoVersion3, typ: 
TypeVarint}, test.Marshaled, &binder)
                if err != nil {
                        t.Errorf("error unmarshaling varint to uint64: %v (test 
#%d)", err, i)
                }
@@ -1545,12 +1545,12 @@ func TestMarshalBigInt(t *testing.T) {
                MarshalError error
        }{
                {
-                       NativeType{proto: 2, typ: TypeBigInt},
+                       NativeType{proto: protoVersion3, typ: TypeBigInt},
                        "-78635384813432117863538481343211",
                        MarshalError("can not marshal string to bigint: 
strconv.ParseInt: parsing \"-78635384813432117863538481343211\": value out of 
range"),
                },
                {
-                       NativeType{proto: 2, typ: TypeBigInt},
+                       NativeType{proto: protoVersion3, typ: TypeBigInt},
                        "922337203685477692259749625974294",
                        MarshalError("can not marshal string to bigint: 
strconv.ParseInt: parsing \"922337203685477692259749625974294\": value out of 
range"),
                },
@@ -1577,13 +1577,9 @@ func equalStringPointerSlice(leftList, rightList 
[]*string) bool {
 }
 
 func TestMarshalList(t *testing.T) {
-       typeInfoV2 := CollectionType{
-               NativeType: NativeType{proto: 2, typ: TypeList},
-               Elem:       NativeType{proto: 2, typ: TypeVarchar},
-       }
        typeInfoV3 := CollectionType{
-               NativeType: NativeType{proto: 3, typ: TypeList},
-               Elem:       NativeType{proto: 3, typ: TypeVarchar},
+               NativeType: NativeType{proto: protoVersion3, typ: TypeList},
+               Elem:       NativeType{proto: protoVersion3, typ: TypeVarchar},
        }
 
        type tc struct {
@@ -1596,37 +1592,6 @@ func TestMarshalList(t *testing.T) {
        valueB := "valueB"
        valueEmpty := ""
        testCases := []tc{
-               {
-                       typeInfo: typeInfoV2,
-                       input:    []*string{&valueA},
-                       expected: []*string{&valueA},
-               },
-               {
-                       typeInfo: typeInfoV2,
-                       input:    []*string{&valueA, &valueB},
-                       expected: []*string{&valueA, &valueB},
-               },
-               {
-                       typeInfo: typeInfoV2,
-                       input:    []*string{&valueA, &valueEmpty, &valueB},
-                       expected: []*string{&valueA, &valueEmpty, &valueB},
-               },
-               {
-                       typeInfo: typeInfoV2,
-                       input:    []*string{&valueEmpty},
-                       expected: []*string{&valueEmpty},
-               },
-               {
-                       // nil values are marshalled to empty values for 
protocol < 3
-                       typeInfo: typeInfoV2,
-                       input:    []*string{nil},
-                       expected: []*string{&valueEmpty},
-               },
-               {
-                       typeInfo: typeInfoV2,
-                       input:    []*string{&valueA, nil, &valueB},
-                       expected: []*string{&valueA, &valueEmpty, &valueB},
-               },
                {
                        typeInfo: typeInfoV3,
                        input:    []*string{&valueEmpty},
@@ -1740,7 +1705,7 @@ func (m *MyPointerMarshaler) MarshalCQL(_ TypeInfo) 
([]byte, error) {
 
 func TestMarshalPointer(t *testing.T) {
        m := &MyPointerMarshaler{}
-       typ := NativeType{proto: 2, typ: TypeInt}
+       typ := NativeType{proto: protoVersion3, typ: TypeInt}
 
        data, err := Marshal(typ, m)
 
@@ -1762,17 +1727,17 @@ func TestMarshalTime(t *testing.T) {
                Value interface{}
        }{
                {
-                       NativeType{proto: 4, typ: TypeTime},
+                       NativeType{proto: protoVersion4, typ: TypeTime},
                        expectedData,
                        duration.Nanoseconds(),
                },
                {
-                       NativeType{proto: 4, typ: TypeTime},
+                       NativeType{proto: protoVersion4, typ: TypeTime},
                        expectedData,
                        duration,
                },
                {
-                       NativeType{proto: 4, typ: TypeTime},
+                       NativeType{proto: protoVersion4, typ: TypeTime},
                        expectedData,
                        &duration,
                },
@@ -1799,46 +1764,46 @@ func TestMarshalTimestamp(t *testing.T) {
                Value interface{}
        }{
                {
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                        time.Date(2013, time.August, 13, 9, 52, 3, 0, time.UTC),
                },
                {
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte("\x00\x00\x01\x40\x77\x16\xe1\xb8"),
                        int64(1376387523000),
                },
                {
                        // 9223372036854 is the maximum time representable in 
ms since the epoch
                        // with int64 if using UnixNano to convert
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte("\x00\x00\x08\x63\x7b\xd0\x5a\xf6"),
                        time.Date(2262, time.April, 11, 23, 47, 16, 854775807, 
time.UTC),
                },
                {
                        // One nanosecond after causes overflow when using 
UnixNano
                        // Instead it should resolve to the same time in ms
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte("\x00\x00\x08\x63\x7b\xd0\x5a\xf6"),
                        time.Date(2262, time.April, 11, 23, 47, 16, 854775808, 
time.UTC),
                },
                {
                        // -9223372036855 is the minimum time representable in 
ms since the epoch
                        // with int64 if using UnixNano to convert
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte("\xff\xff\xf7\x9c\x84\x2f\xa5\x09"),
                        time.Date(1677, time.September, 21, 00, 12, 43, 
145224192, time.UTC),
                },
                {
                        // One nanosecond earlier causes overflow when using 
UnixNano
                        // it should resolve to the same time in ms
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte("\xff\xff\xf7\x9c\x84\x2f\xa5\x09"),
                        time.Date(1677, time.September, 21, 00, 12, 43, 
145224191, time.UTC),
                },
                {
                        // Store the zero time as a blank slice
-                       NativeType{proto: 2, typ: TypeTimestamp},
+                       NativeType{proto: protoVersion3, typ: TypeTimestamp},
                        []byte{},
                        time.Time{},
                },
@@ -1860,10 +1825,10 @@ func TestMarshalTimestamp(t *testing.T) {
 
 func TestMarshalTuple(t *testing.T) {
        info := TupleTypeInfo{
-               NativeType: NativeType{proto: 3, typ: TypeTuple},
+               NativeType: NativeType{proto: protoVersion3, typ: TypeTuple},
                Elems: []TypeInfo{
-                       NativeType{proto: 3, typ: TypeVarchar},
-                       NativeType{proto: 3, typ: TypeVarchar},
+                       NativeType{proto: protoVersion3, typ: TypeVarchar},
+                       NativeType{proto: protoVersion3, typ: TypeVarchar},
                },
        }
 
@@ -2007,10 +1972,10 @@ func TestMarshalTuple(t *testing.T) {
 
 func TestUnmarshalTuple(t *testing.T) {
        info := TupleTypeInfo{
-               NativeType: NativeType{proto: 3, typ: TypeTuple},
+               NativeType: NativeType{proto: protoVersion3, typ: TypeTuple},
                Elems: []TypeInfo{
-                       NativeType{proto: 3, typ: TypeVarchar},
-                       NativeType{proto: 3, typ: TypeVarchar},
+                       NativeType{proto: protoVersion3, typ: TypeVarchar},
+                       NativeType{proto: protoVersion3, typ: TypeVarchar},
                },
        }
 
@@ -2081,10 +2046,10 @@ func TestUnmarshalTuple(t *testing.T) {
 }
 
 func TestMarshalUDTMap(t *testing.T) {
-       typeInfo := UDTTypeInfo{NativeType{proto: 3, typ: TypeUDT}, "", "xyz", 
[]UDTField{
-               {Name: "x", Type: NativeType{proto: 3, typ: TypeInt}},
-               {Name: "y", Type: NativeType{proto: 3, typ: TypeInt}},
-               {Name: "z", Type: NativeType{proto: 3, typ: TypeInt}},
+       typeInfo := UDTTypeInfo{NativeType{proto: protoVersion3, typ: TypeUDT}, 
"", "xyz", []UDTField{
+               {Name: "x", Type: NativeType{proto: protoVersion3, typ: 
TypeInt}},
+               {Name: "y", Type: NativeType{proto: protoVersion3, typ: 
TypeInt}},
+               {Name: "z", Type: NativeType{proto: protoVersion3, typ: 
TypeInt}},
        }}
 
        t.Run("partially bound", func(t *testing.T) {
@@ -2136,10 +2101,10 @@ func TestMarshalUDTMap(t *testing.T) {
 }
 
 func TestMarshalUDTStruct(t *testing.T) {
-       typeInfo := UDTTypeInfo{NativeType{proto: 3, typ: TypeUDT}, "", "xyz", 
[]UDTField{
-               {Name: "x", Type: NativeType{proto: 3, typ: TypeInt}},
-               {Name: "y", Type: NativeType{proto: 3, typ: TypeInt}},
-               {Name: "z", Type: NativeType{proto: 3, typ: TypeInt}},
+       typeInfo := UDTTypeInfo{NativeType{proto: protoVersion3, typ: TypeUDT}, 
"", "xyz", []UDTField{
+               {Name: "x", Type: NativeType{proto: protoVersion3, typ: 
TypeInt}},
+               {Name: "y", Type: NativeType{proto: protoVersion3, typ: 
TypeInt}},
+               {Name: "z", Type: NativeType{proto: protoVersion3, typ: 
TypeInt}},
        }}
 
        type xyzStruct struct {
@@ -2224,7 +2189,7 @@ func TestMarshalNil(t *testing.T) {
        }
 
        for _, typ := range types {
-               data, err := Marshal(NativeType{proto: 3, typ: typ}, nil)
+               data, err := Marshal(NativeType{proto: protoVersion3, typ: 
typ}, nil)
                if err != nil {
                        t.Errorf("unable to marshal nil %v: %v\n", typ, err)
                } else if data != nil {
@@ -2236,7 +2201,7 @@ func TestMarshalNil(t *testing.T) {
 func TestUnmarshalInetCopyBytes(t *testing.T) {
        data := []byte{127, 0, 0, 1}
        var ip net.IP
-       if err := unmarshalInet(NativeType{proto: 2, typ: TypeInet}, data, 
&ip); err != nil {
+       if err := unmarshalInet(NativeType{proto: protoVersion3, typ: 
TypeInet}, data, &ip); err != nil {
                t.Fatal(err)
        }
 
@@ -2250,7 +2215,7 @@ func TestUnmarshalInetCopyBytes(t *testing.T) {
 func TestUnmarshalDate(t *testing.T) {
        data := []uint8{0x80, 0x0, 0x43, 0x31}
        var date time.Time
-       if err := unmarshalDate(NativeType{proto: 2, typ: TypeDate}, data, 
&date); err != nil {
+       if err := unmarshalDate(NativeType{proto: protoVersion3, typ: 
TypeDate}, data, &date); err != nil {
                t.Fatal(err)
        }
 
@@ -2261,7 +2226,7 @@ func TestUnmarshalDate(t *testing.T) {
                return
        }
        var stringDate string
-       if err2 := unmarshalDate(NativeType{proto: 2, typ: TypeDate}, data, 
&stringDate); err2 != nil {
+       if err2 := unmarshalDate(NativeType{proto: protoVersion3, typ: 
TypeDate}, data, &stringDate); err2 != nil {
                t.Fatal(err2)
        }
        if expectedDate != stringDate {
@@ -2281,22 +2246,22 @@ func TestMarshalDate(t *testing.T) {
                Value interface{}
        }{
                {
-                       NativeType{proto: 4, typ: TypeDate},
+                       NativeType{proto: protoVersion4, typ: TypeDate},
                        expectedData,
                        timestamp,
                },
                {
-                       NativeType{proto: 4, typ: TypeDate},
+                       NativeType{proto: protoVersion4, typ: TypeDate},
                        expectedData,
                        now,
                },
                {
-                       NativeType{proto: 4, typ: TypeDate},
+                       NativeType{proto: protoVersion4, typ: TypeDate},
                        expectedData,
                        &now,
                },
                {
-                       NativeType{proto: 4, typ: TypeDate},
+                       NativeType{proto: protoVersion4, typ: TypeDate},
                        expectedData,
                        now.Format("2006-01-02"),
                },
@@ -2340,7 +2305,7 @@ func TestLargeDate(t *testing.T) {
                },
        }
 
-       nativeType := NativeType{proto: 4, typ: TypeDate}
+       nativeType := NativeType{proto: protoVersion4, typ: TypeDate}
 
        for i, test := range marshalDateTests {
                t.Log(i, test)
@@ -2390,22 +2355,22 @@ func TestMarshalDuration(t *testing.T) {
                Value interface{}
        }{
                {
-                       NativeType{proto: 5, typ: TypeDuration},
+                       NativeType{proto: protoVersion5, typ: TypeDuration},
                        expectedData,
                        duration.Nanoseconds(),
                },
                {
-                       NativeType{proto: 5, typ: TypeDuration},
+                       NativeType{proto: protoVersion5, typ: TypeDuration},
                        expectedData,
                        duration,
                },
                {
-                       NativeType{proto: 5, typ: TypeDuration},
+                       NativeType{proto: protoVersion5, typ: TypeDuration},
                        expectedData,
                        durationS,
                },
                {
-                       NativeType{proto: 5, typ: TypeDuration},
+                       NativeType{proto: protoVersion5, typ: TypeDuration},
                        expectedData,
                        &duration,
                },
@@ -2426,13 +2391,9 @@ func TestMarshalDuration(t *testing.T) {
 }
 
 func TestReadCollectionSize(t *testing.T) {
-       listV2 := CollectionType{
-               NativeType: NativeType{proto: 2, typ: TypeList},
-               Elem:       NativeType{proto: 2, typ: TypeVarchar},
-       }
        listV3 := CollectionType{
-               NativeType: NativeType{proto: 3, typ: TypeList},
-               Elem:       NativeType{proto: 3, typ: TypeVarchar},
+               NativeType: NativeType{proto: protoVersion3, typ: TypeList},
+               Elem:       NativeType{proto: protoVersion3, typ: TypeVarchar},
        }
 
        tests := []struct {
@@ -2442,24 +2403,6 @@ func TestReadCollectionSize(t *testing.T) {
                isError      bool
                expectedSize int
        }{
-               {
-                       name:    "short read 0 proto 2",
-                       info:    listV2,
-                       data:    []byte{},
-                       isError: true,
-               },
-               {
-                       name:    "short read 1 proto 2",
-                       info:    listV2,
-                       data:    []byte{0x01},
-                       isError: true,
-               },
-               {
-                       name:         "good read proto 2",
-                       info:         listV2,
-                       data:         []byte{0x01, 0x38},
-                       expectedSize: 0x0138,
-               },
                {
                        name:    "short read 0 proto 3",
                        info:    listV3,
@@ -2493,7 +2436,7 @@ func TestReadCollectionSize(t *testing.T) {
        }
        for _, test := range tests {
                t.Run(test.name, func(t *testing.T) {
-                       size, _, err := readCollectionSize(test.info, test.data)
+                       size, _, err := readCollectionSize(test.data)
                        if test.isError {
                                if err == nil {
                                        t.Fatal("Expected error, but it was 
nil")
@@ -2557,17 +2500,17 @@ func BenchmarkUnmarshalUUID(b *testing.B) {
 
 func TestUnmarshalUDT(t *testing.T) {
        info := UDTTypeInfo{
-               NativeType: NativeType{proto: 4, typ: TypeUDT},
+               NativeType: NativeType{proto: protoVersion4, typ: TypeUDT},
                Name:       "myudt",
                KeySpace:   "myks",
                Elements: []UDTField{
                        {
                                Name: "first",
-                               Type: NativeType{proto: 4, typ: TypeAscii},
+                               Type: NativeType{proto: protoVersion4, typ: 
TypeAscii},
                        },
                        {
                                Name: "second",
-                               Type: NativeType{proto: 4, typ: TypeSmallInt},
+                               Type: NativeType{proto: protoVersion4, typ: 
TypeSmallInt},
                        },
                },
        }
diff --git a/metadata.go b/metadata.go
index c7f8e4b9..c3e6ad33 100644
--- a/metadata.go
+++ b/metadata.go
@@ -32,7 +32,6 @@ import (
        "encoding/hex"
        "encoding/json"
        "fmt"
-       "strconv"
        "strings"
        "sync"
 )
@@ -388,122 +387,7 @@ func compileMetadata(
                table.OrderedColumns = append(table.OrderedColumns, col.Name)
        }
 
-       if protoVersion == protoVersion1 {
-               compileV1Metadata(tables, protoVersion, logger)
-       } else {
-               compileV2Metadata(tables, protoVersion, logger)
-       }
-}
-
-// 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, protoVer int, logger StdLogger) 
{
-       for i := range tables {
-               table := &tables[i]
-
-               // decode the key validator
-               keyValidatorParsed := parseType(table.KeyValidator, 
byte(protoVer), logger)
-               // decode the comparator
-               comparatorParsed := parseType(table.Comparator, byte(protoVer), 
logger)
-
-               // the partition key length is the same as the number of types 
in the
-               // key validator
-               table.PartitionKey = make([]*ColumnMetadata, 
len(keyValidatorParsed.types))
-
-               // V1 protocol only returns "regular" columns from
-               // system.schema_columns (there is no type field for columns)
-               // so the alias information is used to
-               // create the partition key and clustering columns
-
-               // construct the partition key from the alias
-               for i := range table.PartitionKey {
-                       var alias string
-                       if len(table.KeyAliases) > i {
-                               alias = table.KeyAliases[i]
-                       } else if i == 0 {
-                               alias = DEFAULT_KEY_ALIAS
-                       } else {
-                               alias = DEFAULT_KEY_ALIAS + strconv.Itoa(i+1)
-                       }
-
-                       column := &ColumnMetadata{
-                               Keyspace:       table.Keyspace,
-                               Table:          table.Name,
-                               Name:           alias,
-                               Type:           keyValidatorParsed.types[i],
-                               Kind:           ColumnPartitionKey,
-                               ComponentIndex: i,
-                       }
-
-                       table.PartitionKey[i] = column
-                       table.Columns[alias] = column
-               }
-
-               // determine the number of clustering columns
-               size := len(comparatorParsed.types)
-               if comparatorParsed.isComposite {
-                       if len(comparatorParsed.collections) != 0 ||
-                               (len(table.ColumnAliases) == size-1 &&
-                                       comparatorParsed.types[size-1].Type() 
== TypeVarchar) {
-                               size = size - 1
-                       }
-               } else {
-                       if !(len(table.ColumnAliases) != 0 || 
len(table.Columns) == 0) {
-                               size = 0
-                       }
-               }
-
-               table.ClusteringColumns = make([]*ColumnMetadata, size)
-
-               for i := range table.ClusteringColumns {
-                       var alias string
-                       if len(table.ColumnAliases) > i {
-                               alias = table.ColumnAliases[i]
-                       } else if i == 0 {
-                               alias = DEFAULT_COLUMN_ALIAS
-                       } else {
-                               alias = DEFAULT_COLUMN_ALIAS + strconv.Itoa(i+1)
-                       }
-
-                       order := ASC
-                       if comparatorParsed.reversed[i] {
-                               order = DESC
-                       }
-
-                       column := &ColumnMetadata{
-                               Keyspace:       table.Keyspace,
-                               Table:          table.Name,
-                               Name:           alias,
-                               Type:           comparatorParsed.types[i],
-                               Order:          order,
-                               Kind:           ColumnClusteringKey,
-                               ComponentIndex: i,
-                       }
-
-                       table.ClusteringColumns[i] = column
-                       table.Columns[alias] = column
-               }
-
-               if size != len(comparatorParsed.types)-1 {
-                       alias := DEFAULT_VALUE_ALIAS
-                       if len(table.ValueAlias) > 0 {
-                               alias = table.ValueAlias
-                       }
-                       // decode the default validator
-                       defaultValidatorParsed := 
parseType(table.DefaultValidator, byte(protoVer), logger)
-                       column := &ColumnMetadata{
-                               Keyspace: table.Keyspace,
-                               Table:    table.Name,
-                               Name:     alias,
-                               Type:     defaultValidatorParsed.types[0],
-                               Kind:     ColumnRegular,
-                       }
-                       table.Columns[alias] = column
-               }
-       }
+       compileV2Metadata(tables, protoVersion, logger)
 }
 
 // The simpler compile case for V2+ protocol
@@ -611,9 +495,6 @@ func getTableMetadata(session *Session, keyspaceName 
string) ([]TableMetadata, e
                iter *Iter
                scan func(iter *Iter, table *TableMetadata) bool
                stmt string
-
-               keyAliasesJSON    []byte
-               columnAliasesJSON []byte
        )
 
        if session.useSystemSchema { // Cassandra 3.x+
@@ -647,31 +528,6 @@ func getTableMetadata(session *Session, keyspaceName 
string) ([]TableMetadata, e
                        }
                        return r
                }
-       } else if session.cfg.ProtoVersion == protoVersion1 {
-               // we have key aliases
-               stmt = `
-               SELECT
-                       columnfamily_name,
-                       key_validator,
-                       comparator,
-                       default_validator,
-                       key_aliases,
-                       column_aliases,
-                       value_alias
-               FROM system.schema_columnfamilies
-               WHERE keyspace_name = ?`
-
-               scan = func(iter *Iter, table *TableMetadata) bool {
-                       return iter.Scan(
-                               &table.Name,
-                               &table.KeyValidator,
-                               &table.Comparator,
-                               &table.DefaultValidator,
-                               &keyAliasesJSON,
-                               &columnAliasesJSON,
-                               &table.ValueAlias,
-                       )
-               }
        } else {
                stmt = `
                SELECT
@@ -698,34 +554,6 @@ func getTableMetadata(session *Session, keyspaceName 
string) ([]TableMetadata, e
        table := TableMetadata{Keyspace: keyspaceName}
 
        for scan(iter, &table) {
-               var err error
-
-               // decode the key aliases
-               if keyAliasesJSON != nil {
-                       table.KeyAliases = []string{}
-                       err = json.Unmarshal(keyAliasesJSON, &table.KeyAliases)
-                       if err != nil {
-                               iter.Close()
-                               return nil, fmt.Errorf(
-                                       "invalid JSON value '%s' as key_aliases 
for in table '%s': %v",
-                                       keyAliasesJSON, table.Name, err,
-                               )
-                       }
-               }
-
-               // decode the column aliases
-               if columnAliasesJSON != nil {
-                       table.ColumnAliases = []string{}
-                       err = json.Unmarshal(columnAliasesJSON, 
&table.ColumnAliases)
-                       if err != nil {
-                               iter.Close()
-                               return nil, fmt.Errorf(
-                                       "invalid JSON value '%s' as 
column_aliases for in table '%s': %v",
-                                       columnAliasesJSON, table.Name, err,
-                               )
-                       }
-               }
-
                tables = append(tables, table)
                table = TableMetadata{Keyspace: keyspaceName}
        }
@@ -933,9 +761,6 @@ func getTypeInfo(t string, protoVer byte, logger StdLogger) 
TypeInfo {
 }
 
 func getUserTypeMetadata(session *Session, keyspaceName string) 
([]UserTypeMetadata, error) {
-       if session.cfg.ProtoVersion == protoVersion1 {
-               return nil, nil
-       }
        var tableName string
        if session.useSystemSchema {
                tableName = "system_schema.types"
@@ -1179,7 +1004,7 @@ func getMaterializedViewsMetadata(session *Session, 
keyspaceName string) ([]Mate
 }
 
 func getFunctionsMetadata(session *Session, keyspaceName string) 
([]FunctionMetadata, error) {
-       if session.cfg.ProtoVersion == protoVersion1 || 
!session.hasAggregatesAndFunctions {
+       if !session.hasAggregatesAndFunctions {
                return nil, nil
        }
        var tableName string
@@ -1234,7 +1059,7 @@ func getFunctionsMetadata(session *Session, keyspaceName 
string) ([]FunctionMeta
 }
 
 func getAggregatesMetadata(session *Session, keyspaceName string) 
([]AggregateMetadata, error) {
-       if session.cfg.ProtoVersion == protoVersion1 || 
!session.hasAggregatesAndFunctions {
+       if !session.hasAggregatesAndFunctions {
                return nil, nil
        }
        var tableName string
diff --git a/metadata_test.go b/metadata_test.go
index 6b3d1198..21829b33 100644
--- a/metadata_test.go
+++ b/metadata_test.go
@@ -38,310 +38,11 @@ import (
 func TestCompileMetadata(t *testing.T) {
        // V1 tests - these are all based on real examples from the integration 
test ccm cluster
        log := &defaultLogger{}
-       keyspace := &KeyspaceMetadata{
-               Name: "V1Keyspace",
-       }
-       tables := []TableMetadata{
-               {
-                       // This table, found in the system keyspace, has no key 
aliases or column aliases
-                       Keyspace:         "V1Keyspace",
-                       Name:             "Schema",
-                       KeyValidator:     
"org.apache.cassandra.db.marshal.BytesType",
-                       Comparator:       
"org.apache.cassandra.db.marshal.UTF8Type",
-                       DefaultValidator: 
"org.apache.cassandra.db.marshal.BytesType",
-                       KeyAliases:       []string{},
-                       ColumnAliases:    []string{},
-                       ValueAlias:       "",
-               },
-               {
-                       // This table, found in the system keyspace, has key 
aliases, column aliases, and a value alias.
-                       Keyspace:         "V1Keyspace",
-                       Name:             "hints",
-                       KeyValidator:     
"org.apache.cassandra.db.marshal.UUIDType",
-                       Comparator:       
"org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.TimeUUIDType,org.apache.cassandra.db.marshal.Int32Type)",
-                       DefaultValidator: 
"org.apache.cassandra.db.marshal.BytesType",
-                       KeyAliases:       []string{"target_id"},
-                       ColumnAliases:    []string{"hint_id", 
"message_version"},
-                       ValueAlias:       "mutation",
-               },
-               {
-                       // This table, found in the system keyspace, has a 
comparator with collections, but no column aliases
-                       Keyspace:         "V1Keyspace",
-                       Name:             "peers",
-                       KeyValidator:     
"org.apache.cassandra.db.marshal.InetAddressType",
-                       Comparator:       
"org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.ColumnToCollectionType(746f6b656e73:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type)))",
-                       DefaultValidator: 
"org.apache.cassandra.db.marshal.BytesType",
-                       KeyAliases:       []string{"peer"},
-                       ColumnAliases:    []string{},
-                       ValueAlias:       "",
-               },
-               {
-                       // This table, found in the system keyspace, has a 
column alias, but not a composite comparator
-                       Keyspace:         "V1Keyspace",
-                       Name:             "IndexInfo",
-                       KeyValidator:     
"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"},
-                       ValueAlias:       "",
-               },
-               {
-                       // This table, found in the gocql_test keyspace 
following an integration test run, has a composite comparator with collections 
as well as a column alias
-                       Keyspace:         "V1Keyspace",
-                       Name:             "wiki_page",
-                       KeyValidator:     
"org.apache.cassandra.db.marshal.UTF8Type",
-                       Comparator:       
"org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.TimeUUIDType,org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.ColumnToCollectionType(74616773:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type),6174746163686d656e7473:org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type,org.apache.cassandra.db.marshal.BytesType)))",
-                       DefaultValidator: 
"org.apache.cassandra.db.marshal.BytesType",
-                       KeyAliases:       []string{"title"},
-                       ColumnAliases:    []string{"revid"},
-                       ValueAlias:       "",
-               },
-               {
-                       // 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
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "data_center", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.UTF8Type"},
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "host_id", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.UUIDType"},
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "rack", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.UTF8Type"},
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "release_version", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.UTF8Type"},
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "rpc_address", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.InetAddressType"},
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "schema_version", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.UUIDType"},
-               {Keyspace: "V1Keyspace", Table: "peers", Kind: ColumnRegular, 
Name: "tokens", ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type)"},
-       }
-       compileMetadata(1, keyspace, tables, columns, nil, nil, nil, nil, log)
-       assertKeyspaceMetadata(
-               t,
-               keyspace,
-               &KeyspaceMetadata{
-                       Name: "V1Keyspace",
-                       Tables: map[string]*TableMetadata{
-                               "Schema": {
-                                       PartitionKey: []*ColumnMetadata{
-                                               {
-                                                       Name: "key",
-                                                       Type: NativeType{typ: 
TypeBlob},
-                                               },
-                                       },
-                                       ClusteringColumns: []*ColumnMetadata{},
-                                       Columns: map[string]*ColumnMetadata{
-                                               "key": {
-                                                       Name: "key",
-                                                       Type: NativeType{typ: 
TypeBlob},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                       },
-                               },
-                               "hints": {
-                                       PartitionKey: []*ColumnMetadata{
-                                               {
-                                                       Name: "target_id",
-                                                       Type: NativeType{typ: 
TypeUUID},
-                                               },
-                                       },
-                                       ClusteringColumns: []*ColumnMetadata{
-                                               {
-                                                       Name:  "hint_id",
-                                                       Type:  NativeType{typ: 
TypeTimeUUID},
-                                                       Order: ASC,
-                                               },
-                                               {
-                                                       Name:  
"message_version",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                               },
-                                       },
-                                       Columns: map[string]*ColumnMetadata{
-                                               "target_id": {
-                                                       Name: "target_id",
-                                                       Type: NativeType{typ: 
TypeUUID},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                               "hint_id": {
-                                                       Name:  "hint_id",
-                                                       Type:  NativeType{typ: 
TypeTimeUUID},
-                                                       Order: ASC,
-                                                       Kind:  
ColumnClusteringKey,
-                                               },
-                                               "message_version": {
-                                                       Name:  
"message_version",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                                       Kind:  
ColumnClusteringKey,
-                                               },
-                                               "mutation": {
-                                                       Name: "mutation",
-                                                       Type: NativeType{typ: 
TypeBlob},
-                                                       Kind: ColumnRegular,
-                                               },
-                                       },
-                               },
-                               "peers": {
-                                       PartitionKey: []*ColumnMetadata{
-                                               {
-                                                       Name: "peer",
-                                                       Type: NativeType{typ: 
TypeInet},
-                                               },
-                                       },
-                                       ClusteringColumns: []*ColumnMetadata{},
-                                       Columns: map[string]*ColumnMetadata{
-                                               "peer": {
-                                                       Name: "peer",
-                                                       Type: NativeType{typ: 
TypeInet},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                               "data_center":     {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "data_center", 
ComponentIndex: 0, Validator: "org.apache.cassandra.db.marshal.UTF8Type", Type: 
NativeType{typ: TypeVarchar}},
-                                               "host_id":         {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "host_id", 
ComponentIndex: 0, Validator: "org.apache.cassandra.db.marshal.UUIDType", Type: 
NativeType{typ: TypeUUID}},
-                                               "rack":            {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "rack", 
ComponentIndex: 0, Validator: "org.apache.cassandra.db.marshal.UTF8Type", Type: 
NativeType{typ: TypeVarchar}},
-                                               "release_version": {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "release_version", 
ComponentIndex: 0, Validator: "org.apache.cassandra.db.marshal.UTF8Type", Type: 
NativeType{typ: TypeVarchar}},
-                                               "rpc_address":     {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "rpc_address", 
ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.InetAddressType", Type: NativeType{typ: 
TypeInet}},
-                                               "schema_version":  {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "schema_version", 
ComponentIndex: 0, Validator: "org.apache.cassandra.db.marshal.UUIDType", Type: 
NativeType{typ: TypeUUID}},
-                                               "tokens":          {Keyspace: 
"V1Keyspace", Table: "peers", Kind: ColumnRegular, Name: "tokens", 
ComponentIndex: 0, Validator: 
"org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type)",
 Type: CollectionType{NativeType: NativeType{typ: TypeSet}}},
-                                       },
-                               },
-                               "IndexInfo": {
-                                       PartitionKey: []*ColumnMetadata{
-                                               {
-                                                       Name: "table_name",
-                                                       Type: NativeType{typ: 
TypeVarchar},
-                                               },
-                                       },
-                                       ClusteringColumns: []*ColumnMetadata{
-                                               {
-                                                       Name:  "index_name",
-                                                       Type:  NativeType{typ: 
TypeVarchar},
-                                                       Order: DESC,
-                                               },
-                                       },
-                                       Columns: map[string]*ColumnMetadata{
-                                               "table_name": {
-                                                       Name: "table_name",
-                                                       Type: NativeType{typ: 
TypeVarchar},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                               "index_name": {
-                                                       Name:  "index_name",
-                                                       Type:  NativeType{typ: 
TypeVarchar},
-                                                       Order: DESC,
-                                                       Kind:  
ColumnClusteringKey,
-                                               },
-                                               "value": {
-                                                       Name: "value",
-                                                       Type: NativeType{typ: 
TypeBlob},
-                                                       Kind: ColumnRegular,
-                                               },
-                                       },
-                               },
-                               "wiki_page": {
-                                       PartitionKey: []*ColumnMetadata{
-                                               {
-                                                       Name: "title",
-                                                       Type: NativeType{typ: 
TypeVarchar},
-                                               },
-                                       },
-                                       ClusteringColumns: []*ColumnMetadata{
-                                               {
-                                                       Name:  "revid",
-                                                       Type:  NativeType{typ: 
TypeTimeUUID},
-                                                       Order: ASC,
-                                               },
-                                       },
-                                       Columns: map[string]*ColumnMetadata{
-                                               "title": {
-                                                       Name: "title",
-                                                       Type: NativeType{typ: 
TypeVarchar},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                               "revid": {
-                                                       Name: "revid",
-                                                       Type: NativeType{typ: 
TypeTimeUUID},
-                                                       Kind: 
ColumnClusteringKey,
-                                               },
-                                       },
-                               },
-                               "no_names": {
-                                       PartitionKey: []*ColumnMetadata{
-                                               {
-                                                       Name: "key",
-                                                       Type: NativeType{typ: 
TypeUUID},
-                                               },
-                                               {
-                                                       Name: "key2",
-                                                       Type: NativeType{typ: 
TypeUUID},
-                                               },
-                                       },
-                                       ClusteringColumns: []*ColumnMetadata{
-                                               {
-                                                       Name:  "column",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                               },
-                                               {
-                                                       Name:  "column2",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                               },
-                                               {
-                                                       Name:  "column3",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                               },
-                                       },
-                                       Columns: map[string]*ColumnMetadata{
-                                               "key": {
-                                                       Name: "key",
-                                                       Type: NativeType{typ: 
TypeUUID},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                               "key2": {
-                                                       Name: "key2",
-                                                       Type: NativeType{typ: 
TypeUUID},
-                                                       Kind: 
ColumnPartitionKey,
-                                               },
-                                               "column": {
-                                                       Name:  "column",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                                       Kind:  
ColumnClusteringKey,
-                                               },
-                                               "column2": {
-                                                       Name:  "column2",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                                       Kind:  
ColumnClusteringKey,
-                                               },
-                                               "column3": {
-                                                       Name:  "column3",
-                                                       Type:  NativeType{typ: 
TypeInt},
-                                                       Order: ASC,
-                                                       Kind:  
ColumnClusteringKey,
-                                               },
-                                               "value": {
-                                                       Name: "value",
-                                                       Type: NativeType{typ: 
TypeBlob},
-                                                       Kind: ColumnRegular,
-                                               },
-                                       },
-                               },
-                       },
-               },
-       )
-
        // V2 test - V2+ protocol is simpler so here are some toy examples to 
verify that the mapping works
-       keyspace = &KeyspaceMetadata{
+       keyspace := &KeyspaceMetadata{
                Name: "V2Keyspace",
        }
-       tables = []TableMetadata{
+       tables := []TableMetadata{
                {
                        Keyspace: "V2Keyspace",
                        Name:     "Table1",
@@ -351,7 +52,7 @@ func TestCompileMetadata(t *testing.T) {
                        Name:     "Table2",
                },
        }
-       columns = []ColumnMetadata{
+       columns := []ColumnMetadata{
                {
                        Keyspace:       "V2Keyspace",
                        Table:          "Table1",
diff --git a/tuple_test.go b/tuple_test.go
index 296c56fe..de5317b3 100644
--- a/tuple_test.go
+++ b/tuple_test.go
@@ -35,9 +35,6 @@ import (
 func TestTupleSimple(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
 
        err := createTable(session, `CREATE TABLE gocql_test.tuple_test(
                id int,
@@ -79,9 +76,6 @@ func TestTupleSimple(t *testing.T) {
 func TestTuple_NullTuple(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
 
        err := createTable(session, `CREATE TABLE gocql_test.tuple_nil_test(
                id int,
@@ -117,9 +111,6 @@ func TestTuple_NullTuple(t *testing.T) {
 func TestTuple_TupleNotSet(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
 
        err := createTable(session, `CREATE TABLE gocql_test.tuple_not_set_test(
                id int,
@@ -170,9 +161,6 @@ func TestTuple_TupleNotSet(t *testing.T) {
 func TestTupleMapScan(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
 
        err := createTable(session, `CREATE TABLE gocql_test.tuple_map_scan(
                id int,
@@ -203,9 +191,7 @@ func TestTupleMapScan(t *testing.T) {
 func TestTupleMapScanNil(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
+
        err := createTable(session, `CREATE TABLE gocql_test.tuple_map_scan_nil(
                        id int,
                        val frozen<tuple<int, int>>,
@@ -234,9 +220,7 @@ func TestTupleMapScanNil(t *testing.T) {
 func TestTupleMapScanNotSet(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
+
        err := createTable(session, `CREATE TABLE 
gocql_test.tuple_map_scan_not_set(
                        id int,
                        val frozen<tuple<int, int>>,
@@ -266,9 +250,7 @@ func TestTupleLastFieldEmpty(t *testing.T) {
        // Regression test - empty value used to be treated as NULL value in 
the last tuple field
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
+
        err := createTable(session, `CREATE TABLE 
gocql_test.tuple_last_field_empty(
                        id int,
                        val frozen<tuple<text, text>>,
@@ -304,9 +286,6 @@ func TestTupleLastFieldEmpty(t *testing.T) {
 func TestTuple_NestedCollection(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
 
        err := createTable(session, `CREATE TABLE gocql_test.nested_tuples(
                id int,
@@ -356,9 +335,6 @@ func TestTuple_NestedCollection(t *testing.T) {
 func TestTuple_NullableNestedCollection(t *testing.T) {
        session := createSession(t)
        defer session.Close()
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("tuple types are only available of proto>=3")
-       }
 
        err := createTable(session, `CREATE TABLE 
gocql_test.nested_tuples_with_nulls(
                id int,
diff --git a/udt_test.go b/udt_test.go
index f1980f24..65ee10b1 100644
--- a/udt_test.go
+++ b/udt_test.go
@@ -72,10 +72,6 @@ func TestUDT_Marshaler(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.position(
                lat int,
                lon int,
@@ -129,10 +125,6 @@ func TestUDT_Reflect(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.horse(
                name text,
                owner text);`)
@@ -176,22 +168,10 @@ func TestUDT_Reflect(t *testing.T) {
        }
 }
 
-func TestUDT_Proto2error(t *testing.T) {
-       // TODO(zariel): move this to marshal test?
-       _, err := Marshal(NativeType{custom: 
"org.apache.cassandra.db.marshal.UserType.Type", proto: 2}, 1)
-       if err != ErrorUDTUnavailable {
-               t.Fatalf("expected %v got %v", ErrUnavailable, err)
-       }
-}
-
 func TestUDT_NullObject(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.udt_null_type(
                name text,
                owner text);`)
@@ -242,10 +222,6 @@ func TestMapScanUDT(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.log_entry (
                created_timestamp timestamp,
                message text
@@ -329,10 +305,6 @@ func TestUDT_MissingField(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.missing_field(
                name text,
                owner text);`)
@@ -379,10 +351,6 @@ func TestUDT_EmptyCollections(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.nil_collections(
                a list<text>,
                b map<text, text>,
@@ -435,10 +403,6 @@ func TestUDT_UpdateField(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE gocql_test.update_field_udt(
                name text,
                owner text);`)
@@ -492,10 +456,6 @@ func TestUDT_ScanNullUDT(t *testing.T) {
        session := createSession(t)
        defer session.Close()
 
-       if session.cfg.ProtoVersion < protoVersion3 {
-               t.Skip("UDT are only available on protocol >= 3")
-       }
-
        err := createTable(session, `CREATE TYPE 
gocql_test.scan_null_udt_position(
                lat int,
                lon int,


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscr...@cassandra.apache.org
For additional commands, e-mail: commits-h...@cassandra.apache.org

Reply via email to