diff --git a/include/cassandra.h b/include/cassandra.h index 573fd8826..3e1c55cca 100644 --- a/include/cassandra.h +++ b/include/cassandra.h @@ -557,7 +557,8 @@ typedef enum CassColumnType_ { CASS_COLUMN_TYPE_REGULAR, CASS_COLUMN_TYPE_PARTITION_KEY, CASS_COLUMN_TYPE_CLUSTERING_KEY, - CASS_COLUMN_TYPE_STATIC + CASS_COLUMN_TYPE_STATIC, + CASS_COLUMN_TYPE_COMPACT_VALUE } CassColumnType; typedef enum CassErrorSource_ { @@ -2322,6 +2323,9 @@ cass_aggregate_meta_final_func(const CassAggregateMeta* aggregate_meta); /** * Gets the initial condition value for the aggregate. * + * Note: The value of the initial condition will always be + * a "varchar" type for Cassandra 3.0+. + * * @public @memberof CassAggregateMeta * * @param[in] aggregate_meta @@ -4278,6 +4282,18 @@ cass_data_type_set_class_name_n(CassDataType* data_type, const char* class_name, size_t class_name_length); +/** + * Gets the sub-data type count of a UDT (user defined type), tuple + * or collection. + * + * Note: Only valid for UDT, tuple and collection data types. + * + * @param[in] data_type + * @return Returns the number of sub-data types + */ +CASS_EXPORT size_t +cass_data_sub_type_count(const CassDataType* data_type); + /** * Gets the sub-data type of a UDT (user defined type), tuple or collection at * the specified index. diff --git a/src/abstract_data.hpp b/src/abstract_data.hpp index acac4e927..3fd5fad19 100644 --- a/src/abstract_data.hpp +++ b/src/abstract_data.hpp @@ -143,7 +143,7 @@ class AbstractData { protected: virtual size_t get_indices(StringRef name, IndexVec* indices) = 0; - virtual const SharedRefPtr& get_type(size_t index) const = 0; + virtual const DataType::ConstPtr& get_type(size_t index) const = 0; private: template @@ -152,7 +152,7 @@ class AbstractData { return CASS_ERROR_LIB_INDEX_OUT_OF_BOUNDS; } IsValidDataType is_valid_type; - SharedRefPtr data_type(get_type(index)); + DataType::ConstPtr data_type(get_type(index)); if (data_type && !is_valid_type(value, data_type)) { return CASS_ERROR_LIB_INVALID_VALUE_TYPE; } diff --git a/src/collection.hpp b/src/collection.hpp index 1c2dbf337..7863bc132 100644 --- a/src/collection.hpp +++ b/src/collection.hpp @@ -41,7 +41,7 @@ class Collection : public RefCounted { items_.reserve(item_count); } - Collection(const SharedRefPtr& data_type, + Collection(const CollectionType::ConstPtr& data_type, size_t item_count) : data_type_(data_type) { items_.reserve(item_count); @@ -51,7 +51,7 @@ class Collection : public RefCounted { return static_cast(data_type_->value_type()); } - const SharedRefPtr& data_type() const { return data_type_; } + const CollectionType::ConstPtr& data_type() const { return data_type_; } const BufferVec& items() const { return items_; } #define APPEND_TYPE(Type) \ @@ -129,7 +129,7 @@ class Collection : public RefCounted { void encode_items_uint16(char* buf) const; private: - SharedRefPtr data_type_; + CollectionType::ConstPtr data_type_; BufferVec items_; private: diff --git a/src/collection_iterator.cpp b/src/collection_iterator.cpp index 76c92e734..c652c5f3e 100644 --- a/src/collection_iterator.cpp +++ b/src/collection_iterator.cpp @@ -33,7 +33,7 @@ char* CollectionIterator::decode_value(char* position) { int32_t size; char* buffer = decode_size(protocol_version, position, size); - SharedRefPtr data_type; + DataType::ConstPtr data_type; if (collection_->value_type() == CASS_VALUE_TYPE_MAP) { data_type = (index_ % 2 == 0) ? collection_->primary_data_type() : collection_->secondary_data_type(); diff --git a/src/collection_iterator.hpp b/src/collection_iterator.hpp index fbbe2dd30..fee7bb7c4 100644 --- a/src/collection_iterator.hpp +++ b/src/collection_iterator.hpp @@ -67,7 +67,7 @@ class TupleIterator : public ValueIterator { : ValueIterator(CASS_ITERATOR_TYPE_TUPLE) , tuple_(tuple) , position_(tuple->data()) { - SharedRefPtr collection_type(tuple->data_type()); + CollectionType::ConstPtr collection_type(tuple->data_type()); next_ = collection_type->types().begin(); end_ = collection_type->types().end(); } diff --git a/src/control_connection.cpp b/src/control_connection.cpp index 60b1637e8..61df6d3f3 100644 --- a/src/control_connection.cpp +++ b/src/control_connection.cpp @@ -38,12 +38,19 @@ #define SELECT_PEERS "SELECT peer, data_center, rack, release_version, rpc_address FROM system.peers" #define SELECT_PEERS_TOKENS "SELECT peer, data_center, rack, release_version, rpc_address, tokens FROM system.peers" -#define SELECT_KEYSPACES "SELECT * FROM system.schema_keyspaces" -#define SELECT_COLUMN_FAMILIES "SELECT * FROM system.schema_columnfamilies" -#define SELECT_COLUMNS "SELECT * FROM system.schema_columns" -#define SELECT_USERTYPES "SELECT * FROM system.schema_usertypes" -#define SELECT_FUNCTIONS "SELECT * FROM system.schema_functions" -#define SELECT_AGGREGATES "SELECT * FROM system.schema_aggregates" +#define SELECT_KEYSPACES_20 "SELECT * FROM system.schema_keyspaces" +#define SELECT_COLUMN_FAMILIES_20 "SELECT * FROM system.schema_columnfamilies" +#define SELECT_COLUMNS_20 "SELECT * FROM system.schema_columns" +#define SELECT_USERTYPES_21 "SELECT * FROM system.schema_usertypes" +#define SELECT_FUNCTIONS_22 "SELECT * FROM system.schema_functions" +#define SELECT_AGGREGATES_22 "SELECT * FROM system.schema_aggregates" + +#define SELECT_KEYSPACES_30 "SELECT * FROM system_schema.keyspaces" +#define SELECT_TABLES_30 "SELECT * FROM system_schema.tables" +#define SELECT_COLUMNS_30 "SELECT * FROM system_schema.columns" +#define SELECT_USERTYPES_30 "SELECT * FROM system_schema.types" +#define SELECT_FUNCTIONS_30 "SELECT * FROM system_schema.functions" +#define SELECT_AGGREGATES_30 "SELECT * FROM system_schema.aggregates" namespace cass { @@ -194,7 +201,7 @@ void ControlConnection::on_ready(Connection* connection) { // The control connection has to refresh meta when there's a reconnect because // events could have been missed while not connected. - query_meta_all(); + query_meta_hosts(); } void ControlConnection::on_close(Connection* connection) { @@ -359,31 +366,16 @@ void ControlConnection::on_event(EventResponse* response) { } } -//TODO: query and callbacks should be in Metadata -// punting for now because of tight coupling of Session and CC state -void ControlConnection::query_meta_all() { - ScopedRefPtr > handler( - new ControlMultipleRequestHandler(this, ControlConnection::on_query_meta_all, QueryMetadataAllData())); +void ControlConnection::query_meta_hosts() { + ScopedRefPtr > handler( + new ControlMultipleRequestHandler(this, ControlConnection::on_query_hosts, UnusedData())); handler->execute_query(SELECT_LOCAL_TOKENS); handler->execute_query(SELECT_PEERS_TOKENS); - - if (session_->config().use_schema()) { - handler->execute_query(SELECT_KEYSPACES); - handler->execute_query(SELECT_COLUMN_FAMILIES); - handler->execute_query(SELECT_COLUMNS); - if (protocol_version_ >= 3) { - handler->execute_query(SELECT_USERTYPES); - } - if (protocol_version_ >= 4) { - handler->execute_query(SELECT_FUNCTIONS); - handler->execute_query(SELECT_AGGREGATES); - } - } } -void ControlConnection::on_query_meta_all(ControlConnection* control_connection, - const QueryMetadataAllData& unused, - const MultipleRequestHandler::ResponseVec& responses) { +void ControlConnection::on_query_hosts(ControlConnection* control_connection, + const UnusedData& data, + const MultipleRequestHandler::ResponseVec& responses) { Connection* connection = control_connection->connection_; if (connection == NULL) { return; @@ -391,8 +383,6 @@ void ControlConnection::on_query_meta_all(ControlConnection* control_connection, Session* session = control_connection->session_; - session->metadata().clear_and_update_back(); - bool is_initial_connection = (control_connection->state_ == CONTROL_STATE_NEW); // If the 'system.local' table is empty the connection isn't used as a control @@ -460,20 +450,73 @@ void ControlConnection::on_query_meta_all(ControlConnection* control_connection, session->purge_hosts(is_initial_connection); if (session->config().use_schema()) { - session->metadata().update_keyspaces(static_cast(responses[2].get())); - session->metadata().update_tables(static_cast(responses[3].get()), - static_cast(responses[4].get())); - if (control_connection->protocol_version_ >= 3) { - session->metadata().update_user_types(static_cast(responses[5].get())); + control_connection->query_meta_schema(); + } else { + control_connection->state_ = CONTROL_STATE_READY; + session->on_control_connection_ready(); + // Create a new query plan that considers all the new hosts from the + // "system" tables. + control_connection->query_plan_.reset(session->new_query_plan()); + } +} + +//TODO: query and callbacks should be in Metadata +// punting for now because of tight coupling of Session and CC state +void ControlConnection::query_meta_schema() { + ScopedRefPtr > handler( + new ControlMultipleRequestHandler(this, ControlConnection::on_query_meta_schema, UnusedData())); + + if (session_->metadata().cassandra_version() >= VersionNumber(3, 0, 0)) { + handler->execute_query(SELECT_KEYSPACES_30); + handler->execute_query(SELECT_TABLES_30); + handler->execute_query(SELECT_COLUMNS_30); + handler->execute_query(SELECT_USERTYPES_30); + handler->execute_query(SELECT_FUNCTIONS_30); + handler->execute_query(SELECT_AGGREGATES_30); + } else { + handler->execute_query(SELECT_KEYSPACES_20); + handler->execute_query(SELECT_COLUMN_FAMILIES_20); + handler->execute_query(SELECT_COLUMNS_20); + if (session_->metadata().cassandra_version() >= VersionNumber(2, 1, 0)) { + handler->execute_query(SELECT_USERTYPES_21); } - if (control_connection->protocol_version_ >= 4) { - session->metadata().update_functions(static_cast(responses[6].get())); - session->metadata().update_aggregates(static_cast(responses[7].get())); + if (session_->metadata().cassandra_version() >= VersionNumber(2, 2, 0)) { + handler->execute_query(SELECT_FUNCTIONS_22); + handler->execute_query(SELECT_AGGREGATES_22); } - session->metadata().swap_to_back_and_update_front(); - if (control_connection->should_query_tokens_) session->metadata().build(); + } +} + +void ControlConnection::on_query_meta_schema(ControlConnection* control_connection, + const UnusedData& unused, + const MultipleRequestHandler::ResponseVec& responses) { + Connection* connection = control_connection->connection_; + if (connection == NULL) { + return; } + Session* session = control_connection->session_; + + session->metadata().clear_and_update_back(); + + bool is_initial_connection = (control_connection->state_ == CONTROL_STATE_NEW); + + session->metadata().update_keyspaces(static_cast(responses[0].get())); + session->metadata().update_tables(static_cast(responses[1].get()), + static_cast(responses[2].get())); + + if (session->metadata().cassandra_version() >= VersionNumber(2, 1, 0)) { + session->metadata().update_user_types(static_cast(responses[3].get())); + } + + if (session->metadata().cassandra_version() >= VersionNumber(2, 2, 0)) { + session->metadata().update_functions(static_cast(responses[4].get())); + session->metadata().update_aggregates(static_cast(responses[5].get())); + } + + session->metadata().swap_to_back_and_update_front(); + if (control_connection->should_query_tokens_) session->metadata().build(); + if (is_initial_connection) { control_connection->state_ = CONTROL_STATE_READY; session->on_control_connection_ready(); @@ -636,8 +679,9 @@ void ControlConnection::update_node_info(SharedRefPtr host, const Row* row } if (should_query_tokens_) { + bool is_connected_host = connection_ != NULL && host->address().compare(connection_->address()) == 0; std::string partitioner; - if (row->get_string_by_name("partitioner", &partitioner)) { + if (is_connected_host && row->get_string_by_name("partitioner", &partitioner)) { session_->metadata().set_partitioner(partitioner); } v = row->get_by_name("tokens"); @@ -655,7 +699,13 @@ void ControlConnection::update_node_info(SharedRefPtr host, const Row* row } void ControlConnection::refresh_keyspace(const StringRef& keyspace_name) { - std::string query(SELECT_KEYSPACES); + std::string query; + + if (session_->metadata().cassandra_version() >= VersionNumber(3, 0, 0)) { + query.assign(SELECT_KEYSPACES_30); + } else { + query.assign(SELECT_KEYSPACES_20); + } query.append(" WHERE keyspace_name='") .append(keyspace_name.data(), keyspace_name.size()) .append("'"); @@ -683,13 +733,26 @@ void ControlConnection::on_refresh_keyspace(ControlConnection* control_connectio void ControlConnection::refresh_table(const StringRef& keyspace_name, const StringRef& table_name) { - std::string cf_query(SELECT_COLUMN_FAMILIES); - cf_query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) - .append("' AND columnfamily_name='").append(table_name.data(), table_name.size()).append("'"); + std::string cf_query; + std::string col_query; + + if (session_->metadata().cassandra_version() >= VersionNumber(3, 0, 0)) { + cf_query.assign(SELECT_TABLES_30); + cf_query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) + .append("' AND table_name='").append(table_name.data(), table_name.size()).append("'"); - std::string col_query(SELECT_COLUMNS); - col_query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) - .append("' AND columnfamily_name='").append(table_name.data(), table_name.size()).append("'"); + col_query.assign(SELECT_COLUMNS_30); + col_query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) + .append("' AND table_name='").append(table_name.data(), table_name.size()).append("'"); + } else { + cf_query.assign(SELECT_COLUMN_FAMILIES_20); + cf_query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) + .append("' AND columnfamily_name='").append(table_name.data(), table_name.size()).append("'"); + + col_query.assign(SELECT_COLUMNS_20); + col_query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) + .append("' AND columnfamily_name='").append(table_name.data(), table_name.size()).append("'"); + } LOG_DEBUG("Refreshing table %s; %s", cf_query.c_str(), col_query.c_str()); @@ -720,7 +783,13 @@ void ControlConnection::on_refresh_table(ControlConnection* control_connection, void ControlConnection::refresh_type(const StringRef& keyspace_name, const StringRef& type_name) { - std::string query(SELECT_USERTYPES); + std::string query; + if (session_->metadata().cassandra_version() >= VersionNumber(3, 0, 0)) { + query.assign(SELECT_USERTYPES_30); + } else { + query.assign(SELECT_USERTYPES_21); + } + query.append(" WHERE keyspace_name='").append(keyspace_name.data(), keyspace_name.size()) .append("' AND type_name='").append(type_name.data(), type_name.size()).append("'"); @@ -752,12 +821,22 @@ void ControlConnection::refresh_function(const StringRef& keyspace_name, bool is_aggregate) { std::string query; - if (is_aggregate) { - query.assign(SELECT_AGGREGATES); - query.append(" WHERE keyspace_name=? AND aggregate_name=? AND signature=?"); + if (session_->metadata().cassandra_version() >= VersionNumber(3, 0, 0)) { + if (is_aggregate) { + query.assign(SELECT_AGGREGATES_30); + query.append(" WHERE keyspace_name=? AND aggregate_name=? AND argument_types=?"); + } else { + query.assign(SELECT_FUNCTIONS_30); + query.append(" WHERE keyspace_name=? AND function_name=? AND argument_types=?"); + } } else { - query.assign(SELECT_FUNCTIONS); - query.append(" WHERE keyspace_name=? AND function_name=? AND signature=?"); + if (is_aggregate) { + query.assign(SELECT_AGGREGATES_22); + query.append(" WHERE keyspace_name=? AND aggregate_name=? AND signature=?"); + } else { + query.assign(SELECT_FUNCTIONS_22); + query.append(" WHERE keyspace_name=? AND function_name=? AND signature=?"); + } } LOG_DEBUG("Refreshing %s %s in keyspace %s", diff --git a/src/control_connection.hpp b/src/control_connection.hpp index 4a72555e1..472167de3 100644 --- a/src/control_connection.hpp +++ b/src/control_connection.hpp @@ -106,7 +106,7 @@ class ControlConnection : public Connection::Listener { std::string table_name; }; - struct QueryMetadataAllData {}; + struct UnusedData {}; template class ControlHandler : public Handler { @@ -180,28 +180,32 @@ class ControlConnection : public Connection::Listener { virtual void on_availability_change(Connection* connection) {} virtual void on_event(EventResponse* response); - //TODO: possibly reorder callback functions to pair with initiator - static void on_query_meta_all(ControlConnection* control_connection, - const QueryMetadataAllData& data, - const MultipleRequestHandler::ResponseVec& responses); - static void on_refresh_node_info(ControlConnection* control_connection, - const RefreshNodeData& data, - Response* response); - static void on_refresh_node_info_all(ControlConnection* control_connection, - const RefreshNodeData& data, - Response* response); - void on_local_query(ResponseMessage* response); - void on_peer_query(ResponseMessage* response); static void on_reconnect(Timer* timer); bool handle_query_invalid_response(Response* response); void handle_query_failure(CassError code, const std::string& message); void handle_query_timeout(); - void query_meta_all(); + void query_meta_hosts(); + static void on_query_hosts(ControlConnection* control_connection, + const UnusedData& data, + const MultipleRequestHandler::ResponseVec& responses); + + void query_meta_schema(); + static void on_query_meta_schema(ControlConnection* control_connection, + const UnusedData& data, + const MultipleRequestHandler::ResponseVec& responses); + void refresh_node_info(SharedRefPtr host, bool is_new_node, bool query_tokens = false); + static void on_refresh_node_info(ControlConnection* control_connection, + const RefreshNodeData& data, + Response* response); + static void on_refresh_node_info_all(ControlConnection* control_connection, + const RefreshNodeData& data, + Response* response); + void update_node_info(SharedRefPtr host, const Row* row); void refresh_keyspace(const StringRef& keyspace_name); diff --git a/src/data_type.cpp b/src/data_type.cpp index 101850cf4..286f42198 100644 --- a/src/data_type.cpp +++ b/src/data_type.cpp @@ -238,6 +238,19 @@ CassError cass_data_type_set_class_name_n(CassDataType* data_type, return CASS_OK; } +size_t cass_data_sub_type_count(const CassDataType* data_type) { + if (data_type->is_collection() || data_type->is_tuple()) { + const cass::SubTypesDataType* sub_types + = static_cast(data_type->from()); + return sub_types->types().size(); + } else if (data_type->is_user_type()) { + const cass::UserType* user_type + = static_cast(data_type->from()); + return user_type->fields().size(); + } + return 0; +} + CassError cass_data_type_sub_type_name(const CassDataType* data_type, size_t index, const char** name, @@ -360,20 +373,81 @@ void cass_data_type_free(CassDataType* data_type) { namespace cass { -const SharedRefPtr DataType::NIL; +const DataType::ConstPtr DataType::NIL; + +void NativeDataTypes::init_class_names() { + if (!by_class_names_.empty()) return; + by_class_names_["org.apache.cassandra.db.marshal.AsciiType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_ASCII)); + by_class_names_["org.apache.cassandra.db.marshal.BooleanType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_BOOLEAN)); + by_class_names_["org.apache.cassandra.db.marshal.ByteType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TINY_INT)); + by_class_names_["org.apache.cassandra.db.marshal.BytesType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_BLOB)); + by_class_names_["org.apache.cassandra.db.marshal.CounterColumnType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_COUNTER)); + by_class_names_["org.apache.cassandra.db.marshal.DateType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIMESTAMP)); + by_class_names_["org.apache.cassandra.db.marshal.DecimalType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_DECIMAL)); + by_class_names_["org.apache.cassandra.db.marshal.DoubleType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_DOUBLE)); + by_class_names_["org.apache.cassandra.db.marshal.FloatType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_FLOAT)); + by_class_names_["org.apache.cassandra.db.marshal.InetAddressType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_INET)); + by_class_names_["org.apache.cassandra.db.marshal.Int32Type"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_INT)); + by_class_names_["org.apache.cassandra.db.marshal.IntegerType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_INT)); + by_class_names_["org.apache.cassandra.db.marshal.LongType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_BIGINT)); + by_class_names_["org.apache.cassandra.db.marshal.ShortType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_SMALL_INT)); + by_class_names_["org.apache.cassandra.db.marshal.SimpleDateType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_DATE)); + by_class_names_["org.apache.cassandra.db.marshal.TimeType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIME)); + by_class_names_["org.apache.cassandra.db.marshal.TimestampType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIMESTAMP)); + by_class_names_["org.apache.cassandra.db.marshal.TimeUUIDType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIMEUUID)); + by_class_names_["org.apache.cassandra.db.marshal.UTF8Type"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TEXT)); + by_class_names_["org.apache.cassandra.db.marshal.UUIDType"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_UUID)); +} + +const DataType::ConstPtr& NativeDataTypes::by_class_name(const std::string& name) const { + DataTypeMap::const_iterator i = by_class_names_.find(name); + if (i == by_class_names_.end()) return DataType::NIL; + return i->second; +} + +void NativeDataTypes::init_cql_names() { + if (!by_cql_names_.empty()) return; + by_cql_names_["ascii"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_ASCII)); + by_cql_names_["bigint"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_BIGINT)); + by_cql_names_["blob"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_BLOB)); + by_cql_names_["boolean"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_BOOLEAN)); + by_cql_names_["counter"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_COUNTER)); + by_cql_names_["date"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_DATE)); + by_cql_names_["decimal"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_DECIMAL)); + by_cql_names_["double"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_DOUBLE)); + by_cql_names_["float"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_FLOAT)); + by_cql_names_["inet"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_INET)); + by_cql_names_["int"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_INT)); + by_cql_names_["smallint"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_SMALL_INT)); + by_cql_names_["time"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIME)); + by_cql_names_["timestamp"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIMESTAMP)); + by_cql_names_["timeuuid"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TIMEUUID)); + by_cql_names_["tinyint"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TINY_INT)); + by_cql_names_["text"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_TEXT)); + by_cql_names_["uuid"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_UUID)); + by_cql_names_["varchar"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_VARCHAR)); + by_cql_names_["varint"] = DataType::ConstPtr(new DataType(CASS_VALUE_TYPE_VARINT)); +} + +const DataType::ConstPtr& NativeDataTypes::by_cql_name(const std::string& name) const { + DataTypeMap::const_iterator i = by_cql_names_.find(name); + if (i == by_cql_names_.end()) return DataType::NIL; + return i->second; +} + bool cass::IsValidDataType::operator()(const Collection* value, - const SharedRefPtr& data_type) const { + const DataType::ConstPtr& data_type) const { return value->data_type()->equals(data_type); } bool cass::IsValidDataType::operator()(const Tuple* value, - const SharedRefPtr& data_type) const { + const DataType::ConstPtr& data_type) const { return value->data_type()->equals(data_type); } bool cass::IsValidDataType::operator()(const UserTypeValue* value, - const SharedRefPtr& data_type) const { + const DataType::ConstPtr& data_type) const { return value->data_type()->equals(data_type); } diff --git a/src/data_type.hpp b/src/data_type.hpp index 638728ca7..411cee0cb 100644 --- a/src/data_type.hpp +++ b/src/data_type.hpp @@ -65,10 +65,10 @@ inline bool equals_both_not_empty(const std::string& s1, class DataType : public RefCounted { public: - typedef SharedRefPtr Ptr; - typedef std::vector Vec; + typedef SharedRefPtr ConstPtr; + typedef std::vector Vec; - static const SharedRefPtr NIL; + static const DataType::ConstPtr NIL; DataType(CassValueType value_type) : value_type_(value_type) { } @@ -99,7 +99,7 @@ class DataType : public RefCounted { return value_type_ == CASS_VALUE_TYPE_CUSTOM; } - virtual bool equals(const SharedRefPtr& data_type) const { + virtual bool equals(const DataType::ConstPtr& data_type) const { switch (value_type_) { // "text" is an alias for "varchar" case CASS_VALUE_TYPE_TEXT: @@ -168,7 +168,7 @@ class CustomType : public DataType { class_name_ = class_name; } - virtual bool equals(const SharedRefPtr& data_type) const { + virtual bool equals(const DataType::ConstPtr& data_type) const { assert(value_type() == CASS_VALUE_TYPE_CUSTOM); if (data_type->value_type() != CASS_VALUE_TYPE_CUSTOM) { return false; @@ -221,6 +221,8 @@ class SubTypesDataType : public DataType { class CollectionType : public SubTypesDataType { public: + typedef SharedRefPtr ConstPtr; + CollectionType(CassValueType collection_type) : SubTypesDataType(collection_type) { } @@ -233,7 +235,7 @@ class CollectionType : public SubTypesDataType { CollectionType(CassValueType collection_type, const DataType::Vec& types) : SubTypesDataType(collection_type, types) { } - virtual bool equals(const SharedRefPtr& data_type) const { + virtual bool equals(const DataType::ConstPtr& data_type) const { assert(value_type() == CASS_VALUE_TYPE_LIST || value_type() == CASS_VALUE_TYPE_SET || value_type() == CASS_VALUE_TYPE_MAP); @@ -242,7 +244,7 @@ class CollectionType : public SubTypesDataType { return false; } - const SharedRefPtr& collection_type(data_type); + const CollectionType::ConstPtr& collection_type(data_type); // Only compare sub-types if both have sub-types if(!types_.empty() && !collection_type->types_.empty()) { @@ -264,36 +266,38 @@ class CollectionType : public SubTypesDataType { } public: - static SharedRefPtr list(SharedRefPtr element_type) { + static DataType::ConstPtr list(DataType::ConstPtr element_type) { DataType::Vec types; types.push_back(element_type); - return SharedRefPtr(new CollectionType(CASS_VALUE_TYPE_LIST, types)); + return DataType::ConstPtr(new CollectionType(CASS_VALUE_TYPE_LIST, types)); } - static SharedRefPtr set(SharedRefPtr element_type) { + static DataType::ConstPtr set(DataType::ConstPtr element_type) { DataType::Vec types; types.push_back(element_type); - return SharedRefPtr(new CollectionType(CASS_VALUE_TYPE_SET, types)); + return DataType::ConstPtr(new CollectionType(CASS_VALUE_TYPE_SET, types)); } - static SharedRefPtr map(SharedRefPtr key_type, - SharedRefPtr value_type) { + static DataType::ConstPtr map(DataType::ConstPtr key_type, + DataType::ConstPtr value_type) { DataType::Vec types; types.push_back(key_type); types.push_back(value_type); - return SharedRefPtr(new CollectionType(CASS_VALUE_TYPE_MAP, types)); + return DataType::ConstPtr(new CollectionType(CASS_VALUE_TYPE_MAP, types)); } }; class TupleType : public SubTypesDataType { public: + typedef SharedRefPtr ConstPtr; + TupleType() : SubTypesDataType(CASS_VALUE_TYPE_TUPLE) { } TupleType(const DataType::Vec& types) : SubTypesDataType(CASS_VALUE_TYPE_TUPLE, types) { } - virtual bool equals(const SharedRefPtr& data_type) const { + virtual bool equals(const DataType::ConstPtr& data_type) const { assert(value_type() == CASS_VALUE_TYPE_TUPLE); if (value_type() != data_type->value_type()) { @@ -322,17 +326,20 @@ class TupleType : public SubTypesDataType { } }; - class UserType : public DataType { public: + typedef SharedRefPtr Ptr; + typedef SharedRefPtr ConstPtr; + typedef std::map Map; + struct Field : public HashTableEntry { Field(const std::string& field_name, - const SharedRefPtr& type) + const DataType::ConstPtr& type) : name(field_name) , type(type) { } std::string name; - SharedRefPtr type; + DataType::ConstPtr type; }; typedef CaseInsensitiveHashTable::EntryVec FieldVec; @@ -344,6 +351,12 @@ class UserType : public DataType { : DataType(CASS_VALUE_TYPE_UDT) , fields_(field_count) { } + UserType(const std::string& keyspace, + const std::string& type_name ) + : DataType(CASS_VALUE_TYPE_UDT) + , keyspace_(keyspace) + , type_name_(type_name) { } + UserType(const std::string& keyspace, const std::string& type_name, const FieldVec& fields) @@ -370,17 +383,21 @@ class UserType : public DataType { return fields_.get_indices(name, result); } - void add_field(const std::string name, const SharedRefPtr& data_type) { + void add_field(const std::string name, const DataType::ConstPtr& data_type) { fields_.add(Field(name, data_type)); } - virtual bool equals(const SharedRefPtr& data_type) const { + void set_fields(const FieldVec& fields) { + fields_.set_entries(fields); + } + + virtual bool equals(const DataType::ConstPtr& data_type) const { assert(value_type() == CASS_VALUE_TYPE_UDT); if (data_type->value_type() != CASS_VALUE_TYPE_UDT) { return false; } - const SharedRefPtr& user_type(data_type); + const UserType::ConstPtr& user_type(data_type); if (!equals_both_not_empty(keyspace_, user_type->keyspace_)) { return false; @@ -418,120 +435,134 @@ class UserType : public DataType { CaseInsensitiveHashTable fields_; }; +class NativeDataTypes { +public: + void init_class_names(); + const DataType::ConstPtr& by_class_name(const std::string& name) const; + + void init_cql_names(); + const DataType::ConstPtr& by_cql_name(const std::string& name) const; + +private: + typedef std::map DataTypeMap; + DataTypeMap by_class_names_; + DataTypeMap by_cql_names_; +}; + template struct IsValidDataType; template<> struct IsValidDataType { - bool operator()(CassNull, const SharedRefPtr& data_type) const { + bool operator()(CassNull, const DataType::ConstPtr& data_type) const { return true; } }; template<> struct IsValidDataType { - bool operator()(cass_int8_t, const SharedRefPtr& data_type) const { + bool operator()(cass_int8_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_TINY_INT; } }; template<> struct IsValidDataType { - bool operator()(cass_int16_t, const SharedRefPtr& data_type) const { + bool operator()(cass_int16_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_SMALL_INT; } }; template<> struct IsValidDataType { - bool operator()(cass_int32_t, const SharedRefPtr& data_type) const { + bool operator()(cass_int32_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_INT; } }; template<> struct IsValidDataType { - bool operator()(cass_uint32_t, const SharedRefPtr& data_type) const { + bool operator()(cass_uint32_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_DATE; } }; template<> struct IsValidDataType { - bool operator()(cass_int64_t, const SharedRefPtr& data_type) const { + bool operator()(cass_int64_t, const DataType::ConstPtr& data_type) const { return is_int64_type(data_type->value_type()); } }; template<> struct IsValidDataType { - bool operator()(cass_float_t, const SharedRefPtr& data_type) const { + bool operator()(cass_float_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_FLOAT; } }; template<> struct IsValidDataType { - bool operator()(cass_double_t, const SharedRefPtr& data_type) const { + bool operator()(cass_double_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_DOUBLE; } }; template<> struct IsValidDataType { - bool operator()(cass_bool_t, const SharedRefPtr& data_type) const { + bool operator()(cass_bool_t, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_BOOLEAN; } }; template<> struct IsValidDataType { - bool operator()(CassString, const SharedRefPtr& data_type) const { + bool operator()(CassString, const DataType::ConstPtr& data_type) const { return is_string_type(data_type->value_type()); } }; template<> struct IsValidDataType { - bool operator()(CassBytes, const SharedRefPtr& data_type) const { + bool operator()(CassBytes, const DataType::ConstPtr& data_type) const { return is_bytes_type(data_type->value_type()); } }; template<> struct IsValidDataType { - bool operator()(CassUuid, const SharedRefPtr& data_type) const { + bool operator()(CassUuid, const DataType::ConstPtr& data_type) const { return is_uuid_type(data_type->value_type()); } }; template<> struct IsValidDataType { - bool operator()(CassInet, const SharedRefPtr& data_type) const { + bool operator()(CassInet, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_INET; } }; template<> struct IsValidDataType { - bool operator()(CassDecimal, const SharedRefPtr& data_type) const { + bool operator()(CassDecimal, const DataType::ConstPtr& data_type) const { return data_type->value_type() == CASS_VALUE_TYPE_DECIMAL; } }; template<> struct IsValidDataType { - bool operator()(const Collection* value, const SharedRefPtr& data_type) const; + bool operator()(const Collection* value, const DataType::ConstPtr& data_type) const; }; template<> struct IsValidDataType { - bool operator()(const Tuple* value, const SharedRefPtr& data_type) const; + bool operator()(const Tuple* value, const DataType::ConstPtr& data_type) const; }; template<> struct IsValidDataType { - bool operator()(const UserTypeValue* value, const SharedRefPtr& data_type) const; + bool operator()(const UserTypeValue* value, const DataType::ConstPtr& data_type) const; }; } // namespace cass diff --git a/src/type_parser.cpp b/src/data_type_parser.cpp similarity index 54% rename from src/type_parser.cpp rename to src/data_type_parser.cpp index 79dc81494..f233a10d6 100644 --- a/src/type_parser.cpp +++ b/src/data_type_parser.cpp @@ -14,7 +14,7 @@ limitations under the License. */ -#include "type_parser.hpp" +#include "data_type_parser.hpp" #include "utils.hpp" #include "logger.hpp" @@ -34,71 +34,8 @@ #define UDT_TYPE "org.apache.cassandra.db.marshal.UserType" #define TUPLE_TYPE "org.apache.cassandra.db.marshal.TupleType" -#define MARSHAL_PACKAGE "org.apache.cassandra.db.marshal." - namespace cass { -static CassValueType get_value_type(const std::string& str) { - if (starts_with(str, MARSHAL_PACKAGE)) { - StringRef type(StringRef(str).substr(sizeof(MARSHAL_PACKAGE) - 1)); - switch (type.front()) { - case 'A': - if (type == "AsciiType") return CASS_VALUE_TYPE_ASCII; - break; - - case 'B': - if (type == "BooleanType") return CASS_VALUE_TYPE_BOOLEAN; - if (type == "ByteType") return CASS_VALUE_TYPE_TINY_INT; - if (type == "BytesType") return CASS_VALUE_TYPE_BLOB; - break; - - case 'C': - if (type == "CounterColumnType") return CASS_VALUE_TYPE_COUNTER; - break; - - case 'D': - if (type == "DateType") return CASS_VALUE_TYPE_TIMESTAMP; - if (type == "DecimalType") return CASS_VALUE_TYPE_DECIMAL; - if (type == "DoubleType") return CASS_VALUE_TYPE_DOUBLE; - break; - - case 'F': - if (type == "FloatType") return CASS_VALUE_TYPE_FLOAT; - break; - - case 'I': - if (type == "InetAddressType") return CASS_VALUE_TYPE_INET; - if (type == "Int32Type") return CASS_VALUE_TYPE_INT; - if (type == "IntegerType") return CASS_VALUE_TYPE_INT; - break; - - case 'L': - if (type == "LongType") return CASS_VALUE_TYPE_BIGINT; - break; - - case 'S': - if (type == "ShortType") return CASS_VALUE_TYPE_SMALL_INT; - if (type == "SimpleDateType") return CASS_VALUE_TYPE_DATE; - break; - - case 'T': - if (type == "TimeType") return CASS_VALUE_TYPE_TIME; - if (type == "TimestampType") return CASS_VALUE_TYPE_TIMESTAMP; - if (type == "TimeUUIDType") return CASS_VALUE_TYPE_TIMEUUID; - break; - - case 'U': - if (type == "UTF8Type") return CASS_VALUE_TYPE_TEXT; - if (type == "UUIDType") return CASS_VALUE_TYPE_UUID; - break; - - default: - break; - } - } - return CASS_VALUE_TYPE_UNKNOWN; -} - int hex_value(int c) { if (c >= '0' && c <= '9') { return c - '0'; @@ -127,38 +64,218 @@ bool from_hex(const std::string& hex, std::string* result) { return true; } -bool TypeParser::is_reversed(const std::string& type) { +DataType::ConstPtr DataTypeCqlNameParser::parse(const std::string& type, + const NativeDataTypes& native_types, + KeyspaceMetadata* keyspace) { + Parser parser(type, 0); + std::string type_name; + Parser::TypeParamsVec params; + + parser.parse_type_name(&type_name); + std::transform(type_name.begin(), type_name.end(), type_name.begin(), tolower); + + DataType::ConstPtr native_type(native_types.by_cql_name(type_name)); + if (native_type) { + return native_type; + } + + if (type_name == "list") { + parser.parse_type_parameters(¶ms); + if (params.size() != 1) { + LOG_ERROR("Expecting single parameter for list %s", type.c_str()); + return DataType::NIL; + } + DataType::ConstPtr element_type = parse(params[0], native_types, keyspace); + return CollectionType::list(element_type); + } + + if (type_name == "set") { + parser.parse_type_parameters(¶ms); + if (params.size() != 1) { + LOG_ERROR("Expecting single parameter for set %s", type.c_str()); + return DataType::NIL; + } + DataType::ConstPtr element_type = parse(params[0], native_types, keyspace); + return CollectionType::set(element_type); + } + + if (type_name == "map") { + parser.parse_type_parameters(¶ms); + if (params.size() != 2) { + LOG_ERROR("Expecting two parameters for set %s", type.c_str()); + return DataType::NIL; + } + DataType::ConstPtr key_type = parse(params[0], native_types, keyspace); + DataType::ConstPtr value_type = parse(params[1], native_types, keyspace); + return CollectionType::map(key_type, value_type); + } + + if (type_name == "tuple") { + parser.parse_type_parameters(¶ms); + if (params.empty()) { + LOG_ERROR("Expecting at least a one parameter for tuple %s", type.c_str()); + return DataType::NIL; + } + DataType::Vec types; + for (Parser::TypeParamsVec::iterator i = params.begin(), + end = params.end(); + i != end; ++i) { + types.push_back(parse(*i, native_types, keyspace)); + } + return DataType::ConstPtr(new TupleType(types)); + } + + if (type_name == "frozen") { + parser.parse_type_parameters(¶ms); + if (params.size() != 1) { + LOG_ERROR("Expecting single parameter for frozen keyword %s", type.c_str()); + return DataType::NIL; + } + return parse(params[0], native_types, keyspace); + } + + if (type_name == "empty") { + return DataType::ConstPtr(new CustomType(type_name)); + } + + if (type_name.empty()) { + return DataType::NIL; + } + + return keyspace->get_or_create_user_type(type_name); +} + +void DataTypeCqlNameParser::Parser::parse_type_name(std::string* name) { + skip_blank(); + read_next_identifier(name); +} + +void DataTypeCqlNameParser::Parser::parse_type_parameters(TypeParamsVec* params) { + params->clear(); + + if (is_eos()) return; + + skip_blank_and_comma(); + + if (str_[index_] != '<') { + LOG_ERROR("Expecting char %u of %s to be '<' but '%c' found", + (unsigned int)index_, str_.c_str(), str_[index_]); + return; + } + + ++index_; // Skip '<' + + std::string name; + std::string args; + while (skip_blank_and_comma()) { + if (str_[index_] == '>') { + ++index_; + return; + } + parse_type_name(&name); + if (!read_raw_type_parameters(&args)) + return; + params->push_back(name + args); + } +} + +void DataTypeCqlNameParser::Parser::read_next_identifier(std::string* name) { + size_t start_index = index_; + if (str_[start_index] == '"') { + ++index_; + while (!is_eos()) { + bool is_quote = str_[index_] == '"'; + ++index_; + if (is_quote) { + if (!is_eos() && str_[index_] == '"') { + ++index_; + } else { + break; + } + } + } + } else { + while (!is_eos() && (is_identifier_char(str_[index_]) || str_[index_] == '"')) { + ++index_; + } + } + name->assign(str_.begin() + start_index, str_.begin() + index_); +} + +bool DataTypeCqlNameParser::Parser::read_raw_type_parameters(std::string* params) { + skip_blank(); + + params->clear(); + + if (is_eos() || str_[index_] == '>' || str_[index_] == ',') return true; + + if (str_[index_] != '<') { + LOG_ERROR("Expecting char %u of %s to be '<' but '%c' found", + (unsigned int)index_, str_.c_str(), str_[index_]); + return false; + } + + size_t start_index = index_; + int open = 1; + bool in_quotes = false; + while (open > 0) { + ++index_; + + if (is_eos()) { + LOG_ERROR("Angle brackets not closed in type %s", str_.c_str()); + return false; + } + + if (!in_quotes) { + if (str_[index_] == '"') { + in_quotes = true; + } else if (str_[index_] == '<') { + open++; + } else if (str_[index_] == '>') { + open--; + } + } else if (str_[index_] == '"') { + in_quotes = false; + } + } + + ++index_; // Move past the trailing '>' + params->assign(str_.begin() + start_index, str_.begin() + index_); + return true; +} + +bool DataTypeClassNameParser::is_reversed(const std::string& type) { return starts_with(type, REVERSED_TYPE); } -bool TypeParser::is_frozen(const std::string& type) { +bool DataTypeClassNameParser::is_frozen(const std::string& type) { return starts_with(type, FROZEN_TYPE); } -bool TypeParser::is_composite(const std::string& type) { +bool DataTypeClassNameParser::is_composite(const std::string& type) { return starts_with(type, COMPOSITE_TYPE); } -bool TypeParser::is_collection(const std::string& type) { +bool DataTypeClassNameParser::is_collection(const std::string& type) { return starts_with(type, COLLECTION_TYPE); } -bool TypeParser::is_user_type(const std::string& type) { +bool DataTypeClassNameParser::is_user_type(const std::string& type) { return starts_with(type, UDT_TYPE); } -bool TypeParser::is_tuple_type(const std::string& type) { +bool DataTypeClassNameParser::is_tuple_type(const std::string& type) { return starts_with(type, TUPLE_TYPE); } -SharedRefPtr TypeParser::parse_one(const std::string& type) { +DataType::ConstPtr DataTypeClassNameParser::parse_one(const std::string& type, const NativeDataTypes& native_types) { bool frozen = is_frozen(type); std::string class_name; if (is_reversed(type) || frozen) { if (!get_nested_class_name(type, &class_name)) { - return SharedRefPtr(); + return DataType::ConstPtr(); } } else { class_name = type; @@ -172,32 +289,32 @@ SharedRefPtr TypeParser::parse_one(const std::string& type) { if (starts_with(next, LIST_TYPE)) { TypeParamsVec params; if (!parser.get_type_params(¶ms) || params.empty()) { - return SharedRefPtr(); + return DataType::ConstPtr(); } - SharedRefPtr element_type(parse_one(params[0])); + DataType::ConstPtr element_type(parse_one(params[0], native_types)); if (!element_type) { - return SharedRefPtr(); + return DataType::ConstPtr(); } return CollectionType::list(element_type); } else if(starts_with(next, SET_TYPE)) { TypeParamsVec params; if (!parser.get_type_params(¶ms) || params.empty()) { - return SharedRefPtr(); + return DataType::ConstPtr(); } - SharedRefPtr element_type(parse_one(params[0])); + DataType::ConstPtr element_type(parse_one(params[0], native_types)); if (!element_type) { - return SharedRefPtr(); + return DataType::ConstPtr(); } return CollectionType::set(element_type); } else if(starts_with(next, MAP_TYPE)) { TypeParamsVec params; if (!parser.get_type_params(¶ms) || params.size() < 2) { - return SharedRefPtr(); + return DataType::ConstPtr(); } - SharedRefPtr key_type(parse_one(params[0])); - SharedRefPtr value_type(parse_one(params[1])); + DataType::ConstPtr key_type(parse_one(params[0], native_types)); + DataType::ConstPtr value_type(parse_one(params[1], native_types)); if (!key_type || !value_type) { - return SharedRefPtr(); + return DataType::ConstPtr(); } return CollectionType::map(key_type, value_type); } @@ -212,77 +329,80 @@ SharedRefPtr TypeParser::parse_one(const std::string& type) { std::string keyspace; if (!parser.read_one(&keyspace)) { - return SharedRefPtr(); + return DataType::ConstPtr(); } parser.skip_blank_and_comma(); std::string hex; if (!parser.read_one(&hex)) { - return SharedRefPtr(); + return DataType::ConstPtr(); } std::string type_name; if (!from_hex(hex, &type_name)) { LOG_ERROR("Invalid hex string \"%s\" for parameter", hex.c_str()); - return SharedRefPtr(); + return DataType::ConstPtr(); } if (keyspace.empty() || type_name.empty()) { LOG_ERROR("UDT has no keyspace or type name"); - return SharedRefPtr(); + return DataType::ConstPtr(); } parser.skip_blank_and_comma(); NameAndTypeParamsVec raw_fields; if (!parser.get_name_and_type_params(&raw_fields)) { - return SharedRefPtr(); + return DataType::ConstPtr(); } UserType::FieldVec fields; for (NameAndTypeParamsVec::const_iterator i = raw_fields.begin(), end = raw_fields.end(); i != end; ++i) { - SharedRefPtr data_type = parse_one(i->second); + DataType::ConstPtr data_type = parse_one(i->second, native_types); if (!data_type) { - return SharedRefPtr(); + return DataType::ConstPtr(); } fields.push_back(UserType::Field(i->first, data_type)); } - return SharedRefPtr(new UserType(keyspace, type_name, fields)); + return DataType::ConstPtr(new UserType(keyspace, type_name, fields)); } if (is_tuple_type(type)) { TypeParamsVec raw_types; if (!parser.get_type_params(&raw_types)) { - return SharedRefPtr(); + return DataType::ConstPtr(); } DataType::Vec types; for (TypeParamsVec::const_iterator i = raw_types.begin(), end = raw_types.end(); i != end; ++i) { - SharedRefPtr data_type = parse_one(*i); + DataType::ConstPtr data_type = parse_one(*i, native_types); if (!data_type) { - return SharedRefPtr(); + return DataType::ConstPtr(); } types.push_back(data_type); } - return SharedRefPtr(new TupleType(types)); + return DataType::ConstPtr(new TupleType(types)); + } + + DataType::ConstPtr native_type(native_types.by_class_name(next)); + if (native_type) { + return native_type; } - CassValueType t = get_value_type(next); - return t == CASS_VALUE_TYPE_UNKNOWN ? SharedRefPtr(new CustomType(next)) - : SharedRefPtr(new DataType(t)); + return DataType::ConstPtr(new CustomType(next)); } -SharedRefPtr TypeParser::parse_with_composite(const std::string& type) { +SharedRefPtr DataTypeClassNameParser::parse_with_composite(const std::string& type, const NativeDataTypes& native_types) { Parser parser(type, 0); std::string next; parser.get_next_name(&next); if (!is_composite(next)) { - SharedRefPtr data_type = parse_one(type); + DataType::ConstPtr data_type = parse_one(type, native_types); if (!data_type) { return SharedRefPtr(); } @@ -315,7 +435,7 @@ SharedRefPtr TypeParser::parse_with_composite(const std::string& ty for (NameAndTypeParamsVec::const_iterator i = params.begin(), end = params.end(); i != end; ++i) { - SharedRefPtr data_type = parse_one(i->second); + DataType::ConstPtr data_type = parse_one(i->second, native_types); if (!data_type) { return SharedRefPtr(); } @@ -326,7 +446,7 @@ SharedRefPtr TypeParser::parse_with_composite(const std::string& ty DataType::Vec types; ParseResult::ReversedVec reversed; for (size_t i = 0; i < count; ++i) { - SharedRefPtr data_type = parse_one(sub_class_names[i]); + DataType::ConstPtr data_type = parse_one(sub_class_names[i], native_types); if (!data_type) { return SharedRefPtr(); } @@ -337,7 +457,7 @@ SharedRefPtr TypeParser::parse_with_composite(const std::string& ty return SharedRefPtr(new ParseResult(true, types, reversed, collections)); } -bool TypeParser::get_nested_class_name(const std::string& type, std::string* class_name) { +bool DataTypeClassNameParser::get_nested_class_name(const std::string& type, std::string* class_name) { Parser parser(type, 0); parser.get_next_name(); TypeParamsVec params; @@ -349,7 +469,7 @@ bool TypeParser::get_nested_class_name(const std::string& type, std::string* cla return true; } -bool TypeParser::Parser::read_one(std::string* name_and_args) { +bool DataTypeClassNameParser::Parser::read_one(std::string* name_and_args) { std::string name; get_next_name(&name); std::string args; @@ -360,12 +480,12 @@ bool TypeParser::Parser::read_one(std::string* name_and_args) { return true; } -void TypeParser::Parser::get_next_name(std::string* name) { +void DataTypeClassNameParser::Parser::get_next_name(std::string* name) { skip_blank(); read_next_identifier(name); } -bool TypeParser::Parser::get_type_params(TypeParamsVec* params) { +bool DataTypeClassNameParser::Parser::get_type_params(TypeParamsVec* params) { if (is_eos()) { params->clear(); return true; @@ -395,7 +515,7 @@ bool TypeParser::Parser::get_type_params(TypeParamsVec* params) { return false; } -bool TypeParser::Parser::get_name_and_type_params(NameAndTypeParamsVec* params) { +bool DataTypeClassNameParser::Parser::get_name_and_type_params(NameAndTypeParamsVec* params) { while (skip_blank_and_comma()) { if (str_[index_] == ')') { ++index_; @@ -408,7 +528,7 @@ bool TypeParser::Parser::get_name_and_type_params(NameAndTypeParamsVec* params) std::string name; if (!from_hex(hex, &name)) { LOG_ERROR("Invalid hex string \"%s\" for parameter", hex.c_str()); - return SharedRefPtr(); + return DataType::ConstPtr(); } skip_blank(); @@ -434,7 +554,7 @@ bool TypeParser::Parser::get_name_and_type_params(NameAndTypeParamsVec* params) return false; } -bool TypeParser::Parser::get_collection_params(NameAndTypeParamsVec* params) { +bool DataTypeClassNameParser::Parser::get_collection_params(NameAndTypeParamsVec* params) { if (is_eos()) { params->clear(); return true; @@ -450,31 +570,7 @@ bool TypeParser::Parser::get_collection_params(NameAndTypeParamsVec* params) { return get_name_and_type_params(params); } -void TypeParser::Parser::skip_blank() { - while (!is_eos() && is_blank(str_[index_])) { - ++index_; - } -} - -bool TypeParser::Parser::skip_blank_and_comma() { - bool comma_found = false; - while (!is_eos()) { - int c = str_[index_]; - if (c == ',') { - if (comma_found) { - return true; - } else { - comma_found = true; - } - } else if (!is_blank(c)) { - return true; - } - ++index_; - } - return false; -} - -bool TypeParser::Parser::read_raw_arguments(std::string* args) { +bool DataTypeClassNameParser::Parser::read_raw_arguments(std::string* args) { skip_blank(); if (is_eos() || str_[index_] == ')' || str_[index_] == ',') { @@ -509,7 +605,7 @@ bool TypeParser::Parser::read_raw_arguments(std::string* args) { return true; } -void TypeParser::Parser::read_next_identifier(std::string* name) { +void DataTypeClassNameParser::Parser::read_next_identifier(std::string* name) { size_t i = index_; while (!is_eos() && is_identifier_char(str_[index_])) ++index_; @@ -522,7 +618,7 @@ void TypeParser::Parser::read_next_identifier(std::string* name) { } } -void TypeParser::Parser::parse_error(const std::string& str, +void DataTypeClassNameParser::Parser::parse_error(const std::string& str, size_t index, const char* error) { LOG_ERROR("Error parsing '%s' at %u index: %s", diff --git a/src/type_parser.hpp b/src/data_type_parser.hpp similarity index 65% rename from src/type_parser.hpp rename to src/data_type_parser.hpp index cec0aac54..b1e0aaa3e 100644 --- a/src/type_parser.hpp +++ b/src/data_type_parser.hpp @@ -19,6 +19,7 @@ #include "cassandra.h" #include "data_type.hpp" +#include "metadata.hpp" #include "ref_counted.hpp" #include "value.hpp" @@ -28,12 +29,86 @@ namespace cass { +class ParserBase { +public: + ParserBase(const std::string& str, size_t index) + : str_(str) + , index_(index) { } + + void skip() { ++index_; } + + void skip_blank() { + while (!is_eos() && is_blank(str_[index_])) { + ++index_; + } + } + + bool skip_blank_and_comma() { + bool comma_found = false; + while (!is_eos()) { + int c = str_[index_]; + if (c == ',') { + if (comma_found) { + return true; + } else { + comma_found = true; + } + } else if (!is_blank(c)) { + return true; + } + ++index_; + } + return false; + } + + bool is_eos() const { + return index_ >= str_.length(); + } + + static bool is_identifier_char(int c) { + return (c >= '0' && c <= '9') + || (c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') + || c == '-' || c == '+' || c == '.' || c == '_' || c == '&'; + } + + static bool is_blank(int c) { + return c == ' ' || c == '\t' || c == '\n'; + } + +protected: + const std::string str_; + size_t index_; +}; + +class DataTypeCqlNameParser { +public: + static DataType::ConstPtr parse(const std::string& type, + const NativeDataTypes& native_types, + KeyspaceMetadata* keyspace); + +private: + class Parser : public ParserBase { + public: + typedef std::vector TypeParamsVec; + + Parser(const std::string& str, size_t index) + : ParserBase(str, index) { } + + void parse_type_name(std::string* name); + void parse_type_parameters(TypeParamsVec* params); + + private: + void read_next_identifier(std::string* name); + bool read_raw_type_parameters(std::string* param); + }; +}; + class ParseResult : public RefCounted { public: typedef std::vector ReversedVec; - typedef std::map > CollectionMap; + typedef std::map CollectionMap; - ParseResult(SharedRefPtr type, bool reversed) + ParseResult(DataType::ConstPtr type, bool reversed) : is_composite_(false) { types_.push_back(type); reversed_.push_back(reversed); @@ -60,7 +135,7 @@ class ParseResult : public RefCounted { CollectionMap collections_; }; -class TypeParser { +class DataTypeClassNameParser { public: static bool is_reversed(const std::string& type); static bool is_frozen(const std::string& type); @@ -70,8 +145,8 @@ class TypeParser { static bool is_user_type(const std::string& type); static bool is_tuple_type(const std::string& type); - static SharedRefPtr parse_one(const std::string& type); - static SharedRefPtr parse_with_composite(const std::string& type); + static DataType::ConstPtr parse_one(const std::string& type, const NativeDataTypes& native_types); + static SharedRefPtr parse_with_composite(const std::string& type, const NativeDataTypes& native_types); private: static bool get_nested_class_name(const std::string& type, std::string* class_name); @@ -79,15 +154,10 @@ class TypeParser { typedef std::vector TypeParamsVec; typedef std::vector > NameAndTypeParamsVec; - class Parser { + class Parser : public ParserBase { public: Parser(const std::string& str, size_t index) - : str_(str) - , index_(index) {} - - void skip() { ++index_; } - void skip_blank(); - bool skip_blank_and_comma(); + : ParserBase(str, index) { } bool read_one(std::string* name_and_args); void get_next_name(std::string* name = NULL); @@ -102,27 +172,9 @@ class TypeParser { static void parse_error(const std::string& str, size_t index, const char* error); - - bool is_eos() const { - return index_ >= str_.length(); - } - - static bool is_identifier_char(int c) { - return (c >= '0' && c <= '9') - || (c >= 'a' && c <= 'z') || (c >= 'A' && c <= 'Z') - || c == '-' || c == '+' || c == '.' || c == '_' || c == '&'; - } - - static bool is_blank(int c) { - return c == ' ' || c == '\t' || c == '\n'; - } - - private: - const std::string str_; - size_t index_; }; - TypeParser(); + DataTypeClassNameParser(); }; } // namespace cass diff --git a/src/execute_request.hpp b/src/execute_request.hpp index 91e31b610..7aeec814c 100644 --- a/src/execute_request.hpp +++ b/src/execute_request.hpp @@ -50,7 +50,7 @@ class ExecuteRequest : public Statement { return metadata_->get_indices(name, indices); } - virtual const SharedRefPtr& get_type(size_t index) const { + virtual const DataType::ConstPtr& get_type(size_t index) const { return metadata_->get_column_definition(index).data_type; } diff --git a/src/hash_table.hpp b/src/hash_table.hpp index 19c4b40ee..3c70b85f2 100644 --- a/src/hash_table.hpp +++ b/src/hash_table.hpp @@ -89,6 +89,8 @@ class CaseInsensitiveHashTable { size_t add(const T& entry); const EntryVec& entries() const { return entries_; } + void set_entries(const EntryVec& entries); + size_t size() const { return entries_.size(); } private: @@ -114,10 +116,7 @@ CaseInsensitiveHashTable::CaseInsensitiveHashTable(size_t capacity) { template CaseInsensitiveHashTable::CaseInsensitiveHashTable(const EntryVec& entries) { - reset(entries.size()); - for (size_t i = 0; i < entries.size(); ++i) { - add(entries[i]); - } + set_entries(entries); } template @@ -177,6 +176,16 @@ size_t CaseInsensitiveHashTable::add(const T& entry) { return index; } + +template +void CaseInsensitiveHashTable::set_entries(const EntryVec& entries) { + entries_.clear(); + reset(entries.size()); + for (size_t i = 0; i < entries.size(); ++i) { + add(entries[i]); + } +} + template void CaseInsensitiveHashTable::add_index(T* entry) { size_t h = fnv1a_hash_lower(entry->name) & index_mask_; diff --git a/src/host.hpp b/src/host.hpp index 8dba02e46..9f3c4a716 100644 --- a/src/host.hpp +++ b/src/host.hpp @@ -53,6 +53,28 @@ class VersionNumber { , minor_(0) , patch_(0) { } + VersionNumber(int major, int minor, int patch) + : major_(major) + , minor_(minor) + , patch_(patch) { } + + bool operator >=(const VersionNumber& other) const { + return compare(other) >= 0; + } + + int compare(const VersionNumber& other) const { + if (major_ < other.major_) return -1; + if (major_ > other.major_) return 1; + + if (minor_ < other.minor_) return -1; + if (minor_ > other.minor_) return 1; + + if (patch_ < other.patch_) return -1; + if (patch_ > other.patch_) return 1; + + return 0; + } + bool parse(const std::string& version); int major() const { return major_; } diff --git a/src/metadata.cpp b/src/metadata.cpp index b1d7f7189..9af2bb829 100644 --- a/src/metadata.cpp +++ b/src/metadata.cpp @@ -26,7 +26,7 @@ #include "row.hpp" #include "row_iterator.hpp" #include "scoped_lock.hpp" -#include "type_parser.hpp" +#include "data_type_parser.hpp" #include "value.hpp" #include "third_party/rapidjson/rapidjson/document.h" @@ -231,10 +231,6 @@ const CassDataType* cass_column_meta_data_type(const CassColumnMeta* column_meta return CassDataType::to(column_meta->data_type().get()); } -cass_bool_t cass_column_meta_is_reversed(const CassColumnMeta* column_meta) { - return column_meta->is_reversed() ? cass_true : cass_false; -} - const CassValue* cass_column_meta_field_by_name(const CassColumnMeta* column_meta, const char* name) { @@ -505,6 +501,14 @@ const CassValue* cass_iterator_get_meta_field_value(const CassIterator* iterator namespace cass { +static const char* table_column_name(const cass::VersionNumber& cassandra_version) { + return cassandra_version >= VersionNumber(3, 0, 0) ? "table_name" : "columnfamily_name"; +} + +static const char* signature_column_name(const cass::VersionNumber& cassandra_version) { + return cassandra_version >= VersionNumber(3, 0, 0) ? "argument_types" : "signature"; +} + template const T& as_const(const T& x) { return x; } @@ -562,7 +566,7 @@ std::string Metadata::full_function_name(const std::string& name, const StringVe Metadata::SchemaSnapshot Metadata::schema_snapshot() const { ScopedMutex l(&mutex_); return SchemaSnapshot(schema_snapshot_version_, - protocol_version_, + config_.protocol_version, front_.keyspaces()); } @@ -573,9 +577,9 @@ void Metadata::update_keyspaces(ResultResponse* result) { if (is_front_buffer()) { ScopedMutex l(&mutex_); - updating_->update_keyspaces(protocol_version_, result, updates); + updating_->update_keyspaces(config_, result, updates); } else { - updating_->update_keyspaces(protocol_version_, result, updates); + updating_->update_keyspaces(config_, result, updates); } for (KeyspaceMetadata::Map::const_iterator i = updates.begin(); i != updates.end(); ++i) { @@ -588,9 +592,9 @@ void Metadata::update_tables(ResultResponse* tables_result, ResultResponse* colu if (is_front_buffer()) { ScopedMutex l(&mutex_); - updating_->update_tables(protocol_version_, cassandra_version_, tables_result, columns_result); + updating_->update_tables(config_, tables_result, columns_result); } else { - updating_->update_tables(protocol_version_, cassandra_version_, tables_result, columns_result); + updating_->update_tables(config_, tables_result, columns_result); } } @@ -599,9 +603,9 @@ void Metadata::update_user_types(ResultResponse* result) { if (is_front_buffer()) { ScopedMutex l(&mutex_); - updating_->update_user_types(result); + updating_->update_user_types(config_, result); } else { - updating_->update_user_types(result); + updating_->update_user_types(config_, result); } } @@ -610,9 +614,9 @@ void Metadata::update_functions(ResultResponse* result) { if (is_front_buffer()) { ScopedMutex l(&mutex_); - updating_->update_functions(result); + updating_->update_functions(config_, result); } else { - updating_->update_functions(result); + updating_->update_functions(config_, result); } } @@ -621,9 +625,9 @@ void Metadata::update_aggregates(ResultResponse* result) { if (is_front_buffer()) { ScopedMutex l(&mutex_); - updating_->update_aggregates(protocol_version_, result); + updating_->update_aggregates(config_, result); } else { - updating_->update_aggregates(protocol_version_, result); + updating_->update_aggregates(config_, result); } } @@ -683,6 +687,11 @@ void Metadata::drop_aggregate(const std::string& keyspace_name, const std::strin } void Metadata::clear_and_update_back() { + if (config_.cassandra_version >= VersionNumber(3, 0, 0)) { + config_.native_types.init_cql_names(); + } else { + config_.native_types.init_class_names(); + } token_map_.clear(); back_.clear(); updating_ = &back_; @@ -731,7 +740,11 @@ const Value* MetadataBase::add_field(const SharedRefPtr& buffer, cons return value; } -void MetadataBase::add_json_list_field(int version, const Row* row, const std::string& name) { +void MetadataBase::add_field(const SharedRefPtr& buffer, const Value& value, const std::string& name) { + fields_[name] = MetadataField(name, value, buffer); +} + +void MetadataBase::add_json_list_field(int protocol_version, const Row* row, const std::string& name) { const Value* value = row->get_by_name(name); if (value == NULL) return; if (value->size() <= 0) { @@ -764,12 +777,12 @@ void MetadataBase::add_json_list_field(int version, const Row* row, const std::s collection.append(cass::CassString(i->GetString(), i->GetStringLength())); } - size_t encoded_size = collection.get_items_size(version); + size_t encoded_size = collection.get_items_size(protocol_version); SharedRefPtr encoded(RefBuffer::create(encoded_size)); - collection.encode_items(version, encoded->data()); + collection.encode_items(protocol_version, encoded->data()); - Value list(version, + Value list(protocol_version, collection.data_type(), d.Size(), encoded->data(), @@ -777,12 +790,12 @@ void MetadataBase::add_json_list_field(int version, const Row* row, const std::s fields_[name] = MetadataField(name, list, encoded); } -void MetadataBase::add_json_map_field(int version, const Row* row, const std::string& name) { +const Value* MetadataBase::add_json_map_field(int protocol_version, const Row* row, const std::string& name) { const Value* value = row->get_by_name(name); - if (value == NULL) return; + if (value == NULL) return NULL; if (value->size() <= 0) { fields_[name] = MetadataField(name); - return; + return NULL; } int32_t buffer_size = value->size(); @@ -795,13 +808,13 @@ void MetadataBase::add_json_map_field(int version, const Row* row, const std::st if (d.HasParseError()) { LOG_ERROR("Unable to parse JSON (object) for column '%s'", name.c_str()); - return; + return NULL; } if (!d.IsObject()) { LOG_DEBUG("Expected JSON object for column '%s' (probably null or empty)", name.c_str()); fields_[name] = MetadataField(name); - return; + return NULL; } Collection collection(CollectionType::map(SharedRefPtr(new DataType(CASS_VALUE_TYPE_TEXT)), @@ -812,17 +825,18 @@ void MetadataBase::add_json_map_field(int version, const Row* row, const std::st collection.append(CassString(i->value.GetString(), i->value.GetStringLength())); } - size_t encoded_size = collection.get_items_size(version); + size_t encoded_size = collection.get_items_size(protocol_version); SharedRefPtr encoded(RefBuffer::create(encoded_size)); - collection.encode_items(version, encoded->data()); + collection.encode_items(protocol_version, encoded->data()); - Value map(version, + Value map(protocol_version, collection.data_type(), d.MemberCount(), encoded->data(), encoded_size); - fields_[name] = MetadataField(name, map, encoded); + + return (fields_[name] = MetadataField(name, map, encoded)).value(); } const TableMetadata* KeyspaceMetadata::get_table(const std::string& name) const { @@ -848,21 +862,60 @@ void KeyspaceMetadata::drop_table(const std::string& table_name) { tables_->erase(table_name); } +const UserType::Ptr& KeyspaceMetadata::get_or_create_user_type(const std::string& name) { + UserType::Map::iterator i = user_types_->find(name); + if (i == user_types_->end()) { + i = user_types_->insert(std::make_pair(name, + UserType::Ptr(new UserType(MetadataBase::name(), name)))).first; + } + return i->second; +} + const UserType* KeyspaceMetadata::get_user_type(const std::string& name) const { - UserTypeMap::const_iterator i = user_types_->find(name); + UserType::Map::const_iterator i = user_types_->find(name); if (i == user_types_->end()) return NULL; return i->second.get(); } -void KeyspaceMetadata::update(int version, const SharedRefPtr& buffer, const Row* row) { +void KeyspaceMetadata::update(const MetadataConfig& config, const SharedRefPtr& buffer, const Row* row) { add_field(buffer, row, "keyspace_name"); add_field(buffer, row, "durable_writes"); - add_field(buffer, row, "strategy_class"); - add_json_map_field(version, row, "strategy_options"); -} + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + const Value* map = add_field(buffer, row, "replication"); + if (map != NULL && + map->value_type() == CASS_VALUE_TYPE_MAP && + is_string_type(map->primary_value_type()) && + is_string_type(map->secondary_value_type())) { + MapIterator iterator(map); + while (iterator.next()) { + const Value* key = iterator.key(); + const Value* value = iterator.value(); + if (key->to_string_ref() == "class") { + strategy_class_ = value->to_string_ref(); + } + strategy_options_[key->to_string_ref()] = value->to_string_ref(); + } + } + } else { + const Value* value = add_field(buffer, row, "strategy_class"); + if (value != NULL && + is_string_type(value->value_type())) { + strategy_class_ = value->to_string_ref(); + } -void KeyspaceMetadata::add_user_type(const SharedRefPtr& user_type) { - (*user_types_)[user_type->type_name()] = user_type; + const Value* map = add_json_map_field(config.protocol_version, row, "strategy_options"); + if (map != NULL && + map->value_type() == CASS_VALUE_TYPE_MAP && + is_string_type(map->primary_value_type()) && + is_string_type(map->secondary_value_type())) { + MapIterator iterator(map); + while (iterator.next()) { + const Value* key = iterator.key(); + const Value* value = iterator.value(); + strategy_options_[key->to_string_ref()] = value->to_string_ref(); + } + } + } } void KeyspaceMetadata::drop_user_type(const std::string& type_name) { @@ -897,43 +950,56 @@ void KeyspaceMetadata::drop_aggregate(const std::string& full_aggregate_name) { aggregates_->erase(full_aggregate_name); } -TableMetadata::TableMetadata(const std::string& name, - int version, const SharedRefPtr& buffer, const Row* row) +TableMetadata::TableMetadata(const MetadataConfig& config, + const std::string& name, const SharedRefPtr& buffer, const Row* row) : MetadataBase(name) { add_field(buffer, row, "keyspace_name"); - add_field(buffer, row, "columnfamily_name"); - add_field(buffer, row, "cf_id"); + add_field(buffer, row, table_column_name(config.cassandra_version)); add_field(buffer, row, "bloom_filter_fp_chance"); add_field(buffer, row, "caching"); - add_field(buffer, row, "id"); - add_json_list_field(version, row, "column_aliases"); add_field(buffer, row, "comment"); - add_field(buffer, row, "compaction_strategy_class"); - add_json_map_field(version, row, "compaction_strategy_options"); - add_field(buffer, row, "comparator"); - add_json_map_field(version, row, "compression_parameters"); add_field(buffer, row, "default_time_to_live"); - add_field(buffer, row, "default_validator"); - add_field(buffer, row, "dropped_columns"); add_field(buffer, row, "gc_grace_seconds"); - add_field(buffer, row, "index_interval"); - add_field(buffer, row, "is_dense"); - add_field(buffer, row, "key_alias"); - add_json_list_field(version, row, "key_aliases"); - add_field(buffer, row, "key_validator"); - add_field(buffer, row, "local_read_repair_chance"); - add_field(buffer, row, "max_compaction_threshold"); + add_field(buffer, row, "id"); + add_field(buffer, row, "speculative_retry"); add_field(buffer, row, "max_index_interval"); - add_field(buffer, row, "memtable_flush_period_in_ms"); - add_field(buffer, row, "min_compaction_threshold"); add_field(buffer, row, "min_index_interval"); - add_field(buffer, row, "populate_io_cache_on_flush"); + add_field(buffer, row, "memtable_flush_period_in_ms"); add_field(buffer, row, "read_repair_chance"); - add_field(buffer, row, "replicate_on_write"); - add_field(buffer, row, "speculative_retry"); - add_field(buffer, row, "subcomparator"); - add_field(buffer, row, "type"); - add_field(buffer, row, "value_alias"); + + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + add_field(buffer, row, "dclocal_read_repair_chance"); + add_field(buffer, row, "crc_check_chance"); + add_field(buffer, row, "compaction"); + add_field(buffer, row, "compression"); + add_field(buffer, row, "extensions"); + add_field(buffer, row, "flags"); + } else { + add_field(buffer, row, "cf_id"); + add_field(buffer, row, "local_read_repair_chance"); + + add_field(buffer, row, "compaction_strategy_class"); + add_json_map_field(config.protocol_version, row, "compaction_strategy_options"); + add_json_map_field(config.protocol_version, row, "compression_parameters"); + + add_json_list_field(config.protocol_version, row, "column_aliases"); + add_field(buffer, row, "comparator"); + add_field(buffer, row, "subcomparator"); + add_field(buffer, row, "default_validator"); + add_field(buffer, row, "key_alias"); + add_json_list_field(config.protocol_version, row, "key_aliases"); + add_field(buffer, row, "value_alias"); + add_field(buffer, row, "key_validator"); + add_field(buffer, row, "type"); + + add_field(buffer, row, "dropped_columns"); + add_field(buffer, row, "index_interval"); + add_field(buffer, row, "is_dense"); + add_field(buffer, row, "max_compaction_threshold"); + add_field(buffer, row, "min_compaction_threshold"); + add_field(buffer, row, "populate_io_cache_on_flush"); + add_field(buffer, row, "replicate_on_write"); + } } const ColumnMetadata* TableMetadata::get_column(const std::string& name) const { @@ -973,13 +1039,13 @@ size_t get_column_count(const ColumnMetadata::Vec& columns, CassColumnType type) return count; } -void TableMetadata::build_keys_and_sort(const VersionNumber& cassandra_version) { +void TableMetadata::build_keys_and_sort(const MetadataConfig& config) { // Also, Reorders columns so that the order is: // 1) Parition key // 2) Clustering keys // 3) Other columns - if (cassandra_version.major() >= 2) { + if (config.cassandra_version.major() >= 2) { partition_key_.resize(get_column_count(columns_, CASS_COLUMN_TYPE_PARTITION_KEY)); clustering_key_.resize(get_column_count(columns_, CASS_COLUMN_TYPE_CLUSTERING_KEY)); for (ColumnMetadata::Vec::const_iterator i = columns_.begin(), @@ -1012,7 +1078,8 @@ void TableMetadata::build_keys_and_sort(const VersionNumber& cassandra_version) } } - SharedRefPtr key_validator = TypeParser::parse_with_composite(get_string_field("key_validator")); + SharedRefPtr key_validator + = DataTypeClassNameParser::parse_with_composite(get_string_field("key_validator"), config.native_types); size_t size = key_validator->types().size(); partition_key_.reserve(size); for (size_t i = 0; i < size; ++i) { @@ -1045,7 +1112,8 @@ void TableMetadata::build_keys_and_sort(const VersionNumber& cassandra_version) } // TODO: Figure out how to test these special cases and properly document them here - SharedRefPtr comparator = TypeParser::parse_with_composite(get_string_field("comparator")); + SharedRefPtr comparator + = DataTypeClassNameParser::parse_with_composite(get_string_field("comparator"), config.native_types); size_t size = comparator->types().size(); if (comparator->is_composite()) { if (!comparator->collections().empty() || @@ -1087,7 +1155,7 @@ void TableMetadata::build_keys_and_sort(const VersionNumber& cassandra_version) } } -void TableMetadata::key_aliases(KeyAliases* output) const { +void TableMetadata::key_aliases(const NativeDataTypes& native_types, KeyAliases* output) const { const Value* aliases = get_field("key_aliases"); if (aliases != NULL) { output->reserve(aliases->count()); @@ -1097,7 +1165,8 @@ void TableMetadata::key_aliases(KeyAliases* output) const { } } if (output->empty()) {// C* 1.2 tables created via CQL2 or thrift don't have col meta or key aliases - SharedRefPtr key_validator_type = TypeParser::parse_with_composite(get_string_field("key_validator")); + SharedRefPtr key_validator_type + = DataTypeClassNameParser::parse_with_composite(get_string_field("key_validator"), native_types); const size_t count = key_validator_type->types().size(); std::ostringstream ss("key"); for (size_t i = 0; i < count; ++i) { @@ -1110,7 +1179,9 @@ void TableMetadata::key_aliases(KeyAliases* output) const { } } -FunctionMetadata::FunctionMetadata(const std::string& name, const Value* signature, +FunctionMetadata::FunctionMetadata(const MetadataConfig& config, + const std::string& name, const Value* signature, + KeyspaceMetadata* keyspace, const SharedRefPtr& buffer, const Row* row) : MetadataBase(Metadata::full_function_name(name, signature->as_stringlist())) , simple_name_(name) { @@ -1130,18 +1201,31 @@ FunctionMetadata::FunctionMetadata(const std::string& name, const Value* signatu value2->primary_value_type() == CASS_VALUE_TYPE_VARCHAR) { CollectionIterator iterator1(value1); CollectionIterator iterator2(value2); - while (iterator1.next() && iterator2.next()) { - StringRef arg_name(iterator1.value()->to_string_ref()); - DataType::Ptr arg_type(TypeParser::parse_one(iterator2.value()->to_string())); - args_.push_back(Argument(arg_name, arg_type)); - args_by_name_[arg_name] = arg_type; + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + while (iterator1.next() && iterator2.next()) { + StringRef arg_name(iterator1.value()->to_string_ref()); + DataType::ConstPtr arg_type(DataTypeCqlNameParser::parse(iterator2.value()->to_string(), config.native_types, keyspace)); + args_.push_back(Argument(arg_name, arg_type)); + args_by_name_[arg_name] = arg_type; + } + } else { + while (iterator1.next() && iterator2.next()) { + StringRef arg_name(iterator1.value()->to_string_ref()); + DataType::ConstPtr arg_type(DataTypeClassNameParser::parse_one(iterator2.value()->to_string(), config.native_types)); + args_.push_back(Argument(arg_name, arg_type)); + args_by_name_[arg_name] = arg_type; + } } } value1 = add_field(buffer, row, "return_type"); if (value1 != NULL && value1->value_type() == CASS_VALUE_TYPE_VARCHAR) { - return_type_ = TypeParser::parse_one(value1->to_string()); + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + return_type_ = DataTypeCqlNameParser::parse(value1->to_string(), config.native_types, keyspace); + } else { + return_type_ = DataTypeClassNameParser::parse_one(value1->to_string(), config.native_types); + } } value1 = add_field(buffer, row, "body"); @@ -1169,12 +1253,14 @@ const DataType* FunctionMetadata::get_arg_type(StringRef name) const { return i->second.get(); } -AggregateMetadata::AggregateMetadata(const std::string& name, const Value* signature, - const FunctionMetadata::Map& functions, - int version, const SharedRefPtr& buffer, const Row* row) +AggregateMetadata::AggregateMetadata(const MetadataConfig& config, + const std::string& name, const Value* signature, + KeyspaceMetadata* keyspace, + const SharedRefPtr& buffer, const Row* row) : MetadataBase(Metadata::full_function_name(name, signature->as_stringlist())) , simple_name_(name) { const Value* value; + const FunctionMetadata::Map& functions = keyspace->functions(); add_field(buffer, row, "keyspace_name"); add_field(buffer, row, "aggregate_name"); @@ -1184,21 +1270,35 @@ AggregateMetadata::AggregateMetadata(const std::string& name, const Value* signa value->value_type() == CASS_VALUE_TYPE_LIST && value->primary_value_type() == CASS_VALUE_TYPE_VARCHAR) { CollectionIterator iterator(value); - while (iterator.next()) { - arg_types_.push_back(TypeParser::parse_one(iterator.value()->to_string())); + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + while (iterator.next()) { + arg_types_.push_back(DataTypeCqlNameParser::parse(iterator.value()->to_string(), config.native_types, keyspace)); + } + } else { + while (iterator.next()) { + arg_types_.push_back(DataTypeClassNameParser::parse_one(iterator.value()->to_string(), config.native_types)); + } } } value = add_field(buffer, row, "return_type"); if (value != NULL && value->value_type() == CASS_VALUE_TYPE_VARCHAR) { - return_type_ = TypeParser::parse_one(value->to_string()); + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + return_type_ = DataTypeCqlNameParser::parse(value->to_string(), config.native_types, keyspace); + } else { + return_type_ = DataTypeClassNameParser::parse_one(value->to_string(), config.native_types); + } } value = add_field(buffer, row, "state_type"); if (value != NULL && value->value_type() == CASS_VALUE_TYPE_VARCHAR) { - state_type_ = TypeParser::parse_one(value->to_string()); + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + state_type_ = DataTypeCqlNameParser::parse(value->to_string(), config.native_types, keyspace); + } else { + state_type_ = DataTypeClassNameParser::parse_one(value->to_string(), config.native_types); + } } value = add_field(buffer, row, "final_func"); @@ -1226,74 +1326,104 @@ AggregateMetadata::AggregateMetadata(const std::string& name, const Value* signa } value = add_field(buffer, row, "initcond"); - if (value != NULL && - value->value_type() == CASS_VALUE_TYPE_BLOB) { - init_cond_ = Value(version, state_type_, value->data(), value->size()); + if (value != NULL) { + if (value->value_type() == CASS_VALUE_TYPE_BLOB) { + init_cond_ = Value(config.protocol_version, state_type_, value->data(), value->size()); + } else if (config.cassandra_version >= VersionNumber(3, 0, 0) && + value->value_type() == CASS_VALUE_TYPE_VARCHAR) { + init_cond_ = Value(config.protocol_version, + config.native_types.by_cql_name("varchar"), + value->data(), value->size()); + } } } -ColumnMetadata::ColumnMetadata(const std::string& name, - int version, const SharedRefPtr& buffer, const Row* row) +ColumnMetadata::ColumnMetadata(const MetadataConfig& config, + const std::string& name, + KeyspaceMetadata* keyspace, + const SharedRefPtr& buffer, const Row* row) : MetadataBase(name) , type_(CASS_COLUMN_TYPE_REGULAR) - , position_(0) - , is_reversed_(false) { + , position_(0) { const Value* value; add_field(buffer, row, "keyspace_name"); - add_field(buffer, row, "columnfamily_name"); + add_field(buffer, row, table_column_name(config.cassandra_version)); add_field(buffer, row, "column_name"); - value = add_field(buffer, row, "type"); - if (value != NULL && - value->value_type() == CASS_VALUE_TYPE_VARCHAR) { - StringRef type = value->to_string_ref(); - if (type == "partition_key") { - type_ = CASS_COLUMN_TYPE_PARTITION_KEY; - } else if (type == "clustering_key") { - type_ = CASS_COLUMN_TYPE_CLUSTERING_KEY; - } else if (type == "static") { - type_ = CASS_COLUMN_TYPE_STATIC; - } else { - type_ = CASS_COLUMN_TYPE_REGULAR; + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + add_field(buffer, row, "clustering_order"); + add_field(buffer, row, "column_name_bytes"); + + value = add_field(buffer, row, "kind"); + if (value != NULL && + value->value_type() == CASS_VALUE_TYPE_VARCHAR) { + StringRef type = value->to_string_ref(); + if (type == "partition_key") { + type_ = CASS_COLUMN_TYPE_PARTITION_KEY; + } else if (type == "clustering") { + type_ = CASS_COLUMN_TYPE_CLUSTERING_KEY; + } else if (type == "static") { + type_ = CASS_COLUMN_TYPE_STATIC; + } else { + type_ = CASS_COLUMN_TYPE_REGULAR; + } } - } - value = add_field(buffer, row, "component_index"); - // For C* 2.0 to 2.2 this is "null" for single component partition keys - // so the default position of 0 works. C* 1.2 and below don't use this. - if (value != NULL && - value->value_type() == CASS_VALUE_TYPE_INT) { - position_ = value->as_int32(); - } + value = add_field(buffer, row, "position"); + if (value != NULL && + value->value_type() == CASS_VALUE_TYPE_INT) { + position_ = value->as_int32(); + if (position_ < 0) position_ = 0; + } - value = add_field(buffer, row, "validator"); - if (value != NULL && - value->value_type() == CASS_VALUE_TYPE_VARCHAR) { - std::string validator(value->to_string()); - // TODO: Use const changes from CPP-295 - data_type_ = SharedRefPtr(TypeParser::parse_one(validator)); - is_reversed_ = TypeParser::is_reversed(validator); - } + value = add_field(buffer, row, "type"); + if (value != NULL && + value->value_type() == CASS_VALUE_TYPE_VARCHAR) { + std::string type(value->to_string()); + data_type_ = DataTypeCqlNameParser::parse(type, config.native_types, keyspace); + } + } else { + value = add_field(buffer, row, "type"); + if (value != NULL && + value->value_type() == CASS_VALUE_TYPE_VARCHAR) { + StringRef type = value->to_string_ref(); + if (type == "partition_key") { + type_ = CASS_COLUMN_TYPE_PARTITION_KEY; + } else if (type == "clustering_key") { + type_ = CASS_COLUMN_TYPE_CLUSTERING_KEY; + } else if (type == "static") { + type_ = CASS_COLUMN_TYPE_STATIC; + } else if (type == "compact_value") { + type_ = CASS_COLUMN_TYPE_COMPACT_VALUE; + } else { + type_ = CASS_COLUMN_TYPE_REGULAR; + } + } - add_field(buffer, row, "index_name"); - add_json_map_field(version, row, "index_options"); - add_field(buffer, row, "index_type"); -} + value = add_field(buffer, row, "component_index"); + // For C* 2.0 to 2.2 this is "null" for single component partition keys + // so the default position of 0 works. C* 1.2 and below don't use this. + if (value != NULL && + value->value_type() == CASS_VALUE_TYPE_INT) { + position_ = value->as_int32(); + } -void Metadata::SchemaSnapshot::get_table_key_columns(const std::string& ks_name, - const std::string& table_name, - std::vector* output) const { - const KeyspaceMetadata* keyspace = get_keyspace(ks_name); - if (keyspace != NULL) { - const TableMetadata* table = keyspace->get_table(table_name); - if (table != NULL) { - table->key_aliases(output); + value = add_field(buffer, row, "validator"); + if (value != NULL && + value->value_type() == CASS_VALUE_TYPE_VARCHAR) { + std::string validator(value->to_string()); + data_type_ = DataTypeClassNameParser::parse_one(validator, config.native_types); } + + add_field(buffer, row, "index_name"); + add_json_map_field(config.protocol_version, row, "index_options"); + add_field(buffer, row, "index_type"); } } -void Metadata::InternalData::update_keyspaces(int version, ResultResponse* result, KeyspaceMetadata::Map& updates) { +void Metadata::InternalData::update_keyspaces(const MetadataConfig& config, + ResultResponse* result, KeyspaceMetadata::Map& updates) { SharedRefPtr buffer = result->buffer(); result->decode_first_row(); ResultIterator rows(result); @@ -1308,19 +1438,20 @@ void Metadata::InternalData::update_keyspaces(int version, ResultResponse* resul } KeyspaceMetadata* keyspace = get_or_create_keyspace(keyspace_name); - keyspace->update(version, buffer, row); + keyspace->update(config, buffer, row); updates.insert(std::make_pair(keyspace_name, *keyspace)); } } -void Metadata::InternalData::update_tables(int version, const VersionNumber& cassandra_version, ResultResponse* tables_result, ResultResponse* columns_result) { +void Metadata::InternalData::update_tables(const MetadataConfig& config, + ResultResponse* tables_result, ResultResponse* columns_result) { SharedRefPtr buffer = tables_result->buffer(); tables_result->decode_first_row(); ResultIterator rows(tables_result); std::string keyspace_name; - std::string columnfamily_name; + std::string table_name; KeyspaceMetadata* keyspace = NULL; while (rows.next()) { @@ -1328,8 +1459,8 @@ void Metadata::InternalData::update_tables(int version, const VersionNumber& cas const Row* row = rows.row(); if (!row->get_string_by_name("keyspace_name", &temp_keyspace_name) || - !row->get_string_by_name("columnfamily_name", &columnfamily_name)) { - LOG_ERROR("Unable to get column value for 'keyspace_name' or 'columnfamily_name'"); + !row->get_string_by_name(table_column_name(config.cassandra_version), &table_name)) { + LOG_ERROR("Unable to get column value for 'keyspace_name' or '%s'", table_column_name(config.cassandra_version)); continue; } @@ -1338,13 +1469,13 @@ void Metadata::InternalData::update_tables(int version, const VersionNumber& cas keyspace = get_or_create_keyspace(keyspace_name); } - keyspace->add_table(TableMetadata::Ptr(new TableMetadata(columnfamily_name, version, buffer, row))); + keyspace->add_table(TableMetadata::Ptr(new TableMetadata(config, table_name, buffer, row))); } - update_columns(version, cassandra_version, columns_result); + update_columns(config, columns_result); } -void Metadata::InternalData::update_user_types(ResultResponse* result) { +void Metadata::InternalData::update_user_types(const MetadataConfig& config, ResultResponse* result) { result->decode_first_row(); ResultIterator rows(result); @@ -1404,7 +1535,14 @@ void Metadata::InternalData::update_user_types(ResultResponse* result) { std::string field_name(name->to_string()); - SharedRefPtr data_type = TypeParser::parse_one(type->to_string()); + DataType::ConstPtr data_type; + + if (config.cassandra_version >= VersionNumber(3, 0, 0)) { + data_type = DataTypeCqlNameParser::parse(type->to_string(), config.native_types, keyspace); + } else { + data_type = DataTypeClassNameParser::parse_one(type->to_string(), config.native_types); + } + if (!data_type) { LOG_ERROR("Invalid 'field_type' for field \"%s\", keyspace \"%s\" and type \"%s\"", field_name.c_str(), @@ -1416,12 +1554,12 @@ void Metadata::InternalData::update_user_types(ResultResponse* result) { fields.push_back(UserType::Field(field_name, data_type)); } - keyspace->add_user_type( - SharedRefPtr(new UserType(keyspace_name, type_name, fields))); + keyspace->get_or_create_user_type(type_name)->set_fields(fields); } } -void Metadata::InternalData::update_functions(ResultResponse* result) { +void Metadata::InternalData::update_functions(const MetadataConfig& config, + ResultResponse* result) { SharedRefPtr buffer = result->buffer(); result->decode_first_row(); @@ -1435,7 +1573,7 @@ void Metadata::InternalData::update_functions(ResultResponse* result) { std::string function_name; const Row* row = rows.row(); - const Value* signature = row->get_by_name("signature"); + const Value* signature = row->get_by_name(signature_column_name(config.cassandra_version)); if (!row->get_string_by_name("keyspace_name", &temp_keyspace_name) || !row->get_string_by_name("function_name", &function_name) || signature == NULL) { @@ -1448,13 +1586,15 @@ void Metadata::InternalData::update_functions(ResultResponse* result) { keyspace = get_or_create_keyspace(keyspace_name); } - keyspace->add_function(FunctionMetadata::Ptr(new FunctionMetadata(function_name, - signature, buffer, row))); + keyspace->add_function(FunctionMetadata::Ptr(new FunctionMetadata(config, + function_name, signature, + keyspace, + buffer, row))); } } -void Metadata::InternalData::update_aggregates(int version, ResultResponse* result) { +void Metadata::InternalData::update_aggregates(const MetadataConfig& config, ResultResponse* result) { SharedRefPtr buffer = result->buffer(); result->decode_first_row(); @@ -1468,7 +1608,7 @@ void Metadata::InternalData::update_aggregates(int version, ResultResponse* resu std::string aggregate_name; const Row* row = rows.row(); - const Value* signature = row->get_by_name("signature"); + const Value* signature = row->get_by_name(signature_column_name(config.cassandra_version)); if (!row->get_string_by_name("keyspace_name", &temp_keyspace_name) || !row->get_string_by_name("aggregate_name", &aggregate_name) || signature == NULL) { @@ -1481,9 +1621,10 @@ void Metadata::InternalData::update_aggregates(int version, ResultResponse* resu keyspace = get_or_create_keyspace(keyspace_name); } - keyspace->add_aggregate(AggregateMetadata::Ptr(new AggregateMetadata(aggregate_name, signature, - keyspace->functions(), - version, buffer, row))); + keyspace->add_aggregate(AggregateMetadata::Ptr(new AggregateMetadata(config, + aggregate_name, signature, + keyspace, + buffer, row))); } } @@ -1515,14 +1656,14 @@ void Metadata::InternalData::drop_aggregate(const std::string& keyspace_name, co i->second.drop_aggregate(full_aggregate_name); } -void Metadata::InternalData::update_columns(int version, const VersionNumber& cassandra_version, ResultResponse* result) { +void Metadata::InternalData::update_columns(const MetadataConfig& config, ResultResponse* result) { SharedRefPtr buffer = result->buffer(); result->decode_first_row(); ResultIterator rows(result); std::string keyspace_name; - std::string columnfamily_name; + std::string table_name; std::string column_name; KeyspaceMetadata* keyspace = NULL; @@ -1530,13 +1671,14 @@ void Metadata::InternalData::update_columns(int version, const VersionNumber& ca while (rows.next()) { std::string temp_keyspace_name; - std::string temp_columnfamily_name; + std::string temp_table_name; const Row* row = rows.row(); if (!row->get_string_by_name("keyspace_name", &temp_keyspace_name) || - !row->get_string_by_name("columnfamily_name", &temp_columnfamily_name) || + !row->get_string_by_name(table_column_name(config.cassandra_version), &temp_table_name) || !row->get_string_by_name("column_name", &column_name)) { - LOG_ERROR("Unable to get column value for 'keyspace_name', 'columnfamily_name' or 'column_name'"); + LOG_ERROR("Unable to get column value for 'keyspace_name', '%s' or 'column_name'", + table_column_name(config.cassandra_version)); continue; } @@ -1545,22 +1687,23 @@ void Metadata::InternalData::update_columns(int version, const VersionNumber& ca keyspace = get_or_create_keyspace(keyspace_name); } - if (columnfamily_name != temp_columnfamily_name) { + if (table_name != temp_table_name) { // Build keys for the previous table if (table) { - table->build_keys_and_sort(cassandra_version); + table->build_keys_and_sort(config); } - columnfamily_name = temp_columnfamily_name; - table = keyspace->get_or_create_table(columnfamily_name); + table_name = temp_table_name; + table = keyspace->get_or_create_table(table_name); table->clear_columns(); } - table->add_column(ColumnMetadata::Ptr(new ColumnMetadata(column_name, version, buffer, row))); + table->add_column(ColumnMetadata::Ptr(new ColumnMetadata(config, column_name, + keyspace, buffer, row))); } // Build keys for the last table if (table) { - table->build_keys_and_sort(cassandra_version); + table->build_keys_and_sort(config); } } diff --git a/src/metadata.hpp b/src/metadata.hpp index ec39012b4..60004d95b 100644 --- a/src/metadata.hpp +++ b/src/metadata.hpp @@ -24,7 +24,8 @@ #include "scoped_lock.hpp" #include "scoped_ptr.hpp" #include "token_map.hpp" -#include "type_parser.hpp" +#include "data_type.hpp" +#include "value.hpp" #include #include @@ -33,6 +34,7 @@ namespace cass { +class KeyspaceMetadata; class Row; class ResultResponse; @@ -92,6 +94,14 @@ class VecIteratorImpl { typename Collection::const_iterator end_; }; +struct MetadataConfig { + MetadataConfig() + : protocol_version(0) { } + int protocol_version; + VersionNumber cassandra_version; + NativeDataTypes native_types; +}; + class MetadataField { public: typedef std::map Map; @@ -154,8 +164,9 @@ class MetadataBase { protected: const Value* add_field(const SharedRefPtr& buffer, const Row* row, const std::string& name); + void add_field(const SharedRefPtr& buffer, const Value& value, const std::string& name); void add_json_list_field(int version, const Row* row, const std::string& name); - void add_json_map_field(int version, const Row* row, const std::string& name); + const Value* add_json_map_field(int version, const Row* row, const std::string& name); MetadataField::Map fields_; @@ -186,21 +197,23 @@ class FunctionMetadata : public MetadataBase, public RefCounted Vec; - typedef std::map Map; + typedef std::map Map; - Argument(const StringRef& name, const DataType::Ptr& type) + Argument(const StringRef& name, const DataType::ConstPtr& type) : name(name) , type(type) { } StringRef name; - DataType::Ptr type; + DataType::ConstPtr type; }; - FunctionMetadata(const std::string& name, const Value* signature, + FunctionMetadata(const MetadataConfig& config, + const std::string& name, const Value* signature, + KeyspaceMetadata* keyspace, const SharedRefPtr& buffer, const Row* row); const std::string& simple_name() const { return simple_name_; } const Argument::Vec& args() const { return args_; } - const DataType::Ptr& return_type() const { return return_type_; } + const DataType::ConstPtr& return_type() const { return return_type_; } StringRef body() const { return body_; } StringRef language() const { return language_; } bool called_on_null_input() const { return called_on_null_input_; } @@ -211,7 +224,7 @@ class FunctionMetadata : public MetadataBase, public RefCounted Map; typedef std::vector Vec; - AggregateMetadata(const std::string& name, const Value* signature, - const FunctionMetadata::Map& functions, - int version, const SharedRefPtr& buffer, const Row* row); + AggregateMetadata(const MetadataConfig& config, + const std::string& name, const Value* signature, + KeyspaceMetadata* keyspace, + const SharedRefPtr& buffer, const Row* row); const std::string& simple_name() const { return simple_name_; } const DataType::Vec arg_types() const { return arg_types_; } - const DataType::Ptr& return_type() const { return return_type_; } - const DataType::Ptr& state_type() const { return state_type_; } + const DataType::ConstPtr& return_type() const { return return_type_; } + const DataType::ConstPtr& state_type() const { return state_type_; } const FunctionMetadata::Ptr& state_func() const { return state_func_; } const FunctionMetadata::Ptr& final_func() const { return final_func_; } const Value& init_cond() const { return init_cond_; } @@ -238,8 +252,8 @@ class AggregateMetadata : public MetadataBase, public RefCounted { ColumnMetadata(const std::string& name) : MetadataBase(name) , type_(CASS_COLUMN_TYPE_REGULAR) - , position_(0) - , is_reversed_(false) { } + , position_(0) { } ColumnMetadata(const std::string& name, int32_t position, CassColumnType type, - const SharedRefPtr& data_type) + const DataType::ConstPtr& data_type) : MetadataBase(name) , type_(type) , position_(position) - , data_type_(data_type) - , is_reversed_(false) { } + , data_type_(data_type) { } - ColumnMetadata(const std::string& name, - int version, const SharedRefPtr& buffer, const Row* row); + ColumnMetadata(const MetadataConfig& config, + const std::string& name, + KeyspaceMetadata* keyspace, + const SharedRefPtr& buffer, const Row* row); CassColumnType type() const { return type_; } int32_t position() const { return position_; } - const SharedRefPtr& data_type() const { return data_type_; } - bool is_reversed() const { return is_reversed_; } - + const DataType::ConstPtr& data_type() const { return data_type_; } private: CassColumnType type_; int32_t position_; - SharedRefPtr data_type_; - bool is_reversed_; + DataType::ConstPtr data_type_; private: DISALLOW_COPY_AND_ASSIGN(ColumnMetadata); @@ -302,8 +313,8 @@ class TableMetadata : public MetadataBase, public RefCounted { TableMetadata(const std::string& name) : MetadataBase(name) { } - TableMetadata(const std::string& name, - int version, const SharedRefPtr& buffer, const Row* row); + TableMetadata(const MetadataConfig& config, + const std::string& name, const SharedRefPtr& buffer, const Row* row); const ColumnMetadata::Vec& columns() const { return columns_; } const ColumnMetadata::Vec& partition_key() const { return partition_key_; } @@ -314,8 +325,8 @@ class TableMetadata : public MetadataBase, public RefCounted { const ColumnMetadata::Ptr& get_or_create_column(const std::string& name); void add_column(const ColumnMetadata::Ptr& column); void clear_columns(); - void build_keys_and_sort(const VersionNumber& cassandra_version); - void key_aliases(KeyAliases* output) const; + void build_keys_and_sort(const MetadataConfig& config); + void key_aliases(const NativeDataTypes& native_types, KeyAliases* output) const; private: ColumnMetadata::Vec columns_; @@ -331,7 +342,7 @@ class KeyspaceMetadata : public MetadataBase { public: typedef std::map Map; typedef CopyOnWritePtr MapPtr; - typedef std::map > UserTypeMap; + typedef std::map OptionsMap; class TableIterator : public MetadataIteratorImpl > { public: @@ -340,10 +351,10 @@ class KeyspaceMetadata : public MetadataBase { const TableMetadata* table() const { return impl_.item().get(); } }; - class TypeIterator : public MetadataIteratorImpl > > { + class TypeIterator : public MetadataIteratorImpl > { public: TypeIterator(const TypeIterator::Collection& collection) - : MetadataIteratorImpl > >(CASS_ITERATOR_TYPE_TYPE_META, collection) { } + : MetadataIteratorImpl >(CASS_ITERATOR_TYPE_TYPE_META, collection) { } const UserType* type() const { return impl_.item().get(); } }; @@ -364,13 +375,15 @@ class KeyspaceMetadata : public MetadataBase { KeyspaceMetadata(const std::string& name) : MetadataBase(name) , tables_(new TableMetadata::Map) - , user_types_(new UserTypeMap) + , user_types_(new UserType::Map) , functions_(new FunctionMetadata::Map) , aggregates_(new AggregateMetadata::Map) { } - void update(int version, const SharedRefPtr& buffer, const Row* row); + void update(const MetadataConfig& config, + const SharedRefPtr& buffer, const Row* row); const FunctionMetadata::Map& functions() const { return *functions_; } + const UserType::Map& user_types() const { return *user_types_; } Iterator* iterator_tables() const { return new TableIterator(*tables_); } const TableMetadata* get_table(const std::string& table_name) const; @@ -380,7 +393,7 @@ class KeyspaceMetadata : public MetadataBase { Iterator* iterator_user_types() const { return new TypeIterator(*user_types_); } const UserType* get_user_type(const std::string& type_name) const; - void add_user_type(const SharedRefPtr& user_type); + const UserType::Ptr& get_or_create_user_type(const std::string& name); void drop_user_type(const std::string& type_name); Iterator* iterator_functions() const { return new FunctionIterator(*functions_); } @@ -393,12 +406,15 @@ class KeyspaceMetadata : public MetadataBase { void add_aggregate(const AggregateMetadata::Ptr& aggregate); void drop_aggregate(const std::string& full_aggregate_name); - std::string strategy_class() const { return get_string_field("strategy_class"); } - const Value* strategy_options() const { return get_field("strategy_options"); } + StringRef strategy_class() const { return strategy_class_; } + const OptionsMap& strategy_options() const { return strategy_options_; } private: + StringRef strategy_class_; + OptionsMap strategy_options_; + CopyOnWritePtr tables_; - CopyOnWritePtr user_types_; + CopyOnWritePtr user_types_; CopyOnWritePtr functions_; CopyOnWritePtr aggregates_; }; @@ -430,10 +446,6 @@ class Metadata { const UserType* get_user_type(const std::string& keyspace_name, const std::string& type_name) const; - void get_table_key_columns(const std::string& ks_name, - const std::string& cf_name, - std::vector* output) const; - private: uint32_t version_; int protocol_version_; @@ -445,8 +457,7 @@ class Metadata { public: Metadata() : updating_(&front_) - , schema_snapshot_version_(0) - , protocol_version_(0) { + , schema_snapshot_version_(0) { uv_mutex_init(&mutex_); } @@ -479,11 +490,12 @@ class Metadata { void clear(); void set_protocol_version(int version) { - protocol_version_ = version; + config_.protocol_version = version; } + const VersionNumber& cassandra_version() const { return config_.cassandra_version; } void set_cassandra_version(const VersionNumber& cassandra_version) { - cassandra_version_ = cassandra_version; + config_.cassandra_version = cassandra_version; } void set_partitioner(const std::string& partitioner_class) { token_map_.set_partitioner(partitioner_class); } @@ -504,11 +516,11 @@ class Metadata { const KeyspaceMetadata::MapPtr& keyspaces() const { return keyspaces_; } - void update_keyspaces(int version, ResultResponse* result, KeyspaceMetadata::Map& updates); - void update_tables(int version, const VersionNumber& casandra_version, ResultResponse* tables_result, ResultResponse* columns_result); - void update_user_types(ResultResponse* result); - void update_functions(ResultResponse* result); - void update_aggregates(int version, ResultResponse* result); + void update_keyspaces(const MetadataConfig& config, ResultResponse* result, KeyspaceMetadata::Map& updates); + void update_tables(const MetadataConfig& config, ResultResponse* tables_result, ResultResponse* columns_result); + void update_user_types(const MetadataConfig& config, ResultResponse* result); + void update_functions(const MetadataConfig& config, ResultResponse* result); + void update_aggregates(const MetadataConfig& config, ResultResponse* result); void drop_keyspace(const std::string& keyspace_name); void drop_table(const std::string& keyspace_name, const std::string& table_name); @@ -525,7 +537,7 @@ class Metadata { } private: - void update_columns(int version, const VersionNumber& cassandra_version, ResultResponse* result); + void update_columns(const MetadataConfig& config, ResultResponse* result); KeyspaceMetadata* get_or_create_keyspace(const std::string& name); @@ -552,9 +564,7 @@ class Metadata { // Only used internally on a single thread, there's // no need for copy-on-write. - int protocol_version_; - VersionNumber cassandra_version_; - + MetadataConfig config_; private: DISALLOW_COPY_AND_ASSIGN(Metadata); diff --git a/src/query_request.hpp b/src/query_request.hpp index da6a71c86..8fe546a14 100644 --- a/src/query_request.hpp +++ b/src/query_request.hpp @@ -66,7 +66,7 @@ class QueryRequest : public Statement { virtual size_t get_indices(StringRef name, IndexVec* indices); - virtual const SharedRefPtr& get_type(size_t index) const { + virtual const DataType::ConstPtr& get_type(size_t index) const { return DataType::NIL; } diff --git a/src/replication_strategy.cpp b/src/replication_strategy.cpp index ed5f09859..3ed91e18b 100644 --- a/src/replication_strategy.cpp +++ b/src/replication_strategy.cpp @@ -27,32 +27,54 @@ namespace cass { +static void build_dc_replicas(const KeyspaceMetadata::OptionsMap& strategy_options, + NetworkTopologyStrategy::DCReplicaCountMap* output) { + for (KeyspaceMetadata::OptionsMap::const_iterator i = strategy_options.begin(), + end = strategy_options.end(); + i != end; ++i) { + if (i->first != "class") { + size_t replication_factor = strtoul(i->second.to_string().c_str(), NULL, 10); + if (replication_factor > 0) { + (*output)[i->first.to_string()] = replication_factor; + } + } + } +} + +static size_t get_replication_factor(const KeyspaceMetadata::OptionsMap& strategy_options) { + size_t replication_factor = 0; + KeyspaceMetadata::OptionsMap::const_iterator i = strategy_options.find("replication_factor"); + if (i != strategy_options.end()) { + replication_factor = strtoul(i->second.to_string().c_str(), NULL, 10); + } + if (replication_factor == 0) { + LOG_WARN("Replication factor of 0"); + } + return replication_factor; +} + + SharedRefPtr ReplicationStrategy::from_keyspace_meta(const KeyspaceMetadata& ks_meta) { - std::string strategy_class = ks_meta.strategy_class(); + StringRef strategy_class = ks_meta.strategy_class(); SharedRefPtr strategy; if (ends_with(strategy_class, NetworkTopologyStrategy::STRATEGY_CLASS)) { - return SharedRefPtr( - new NetworkTopologyStrategy(strategy_class, ks_meta.strategy_options())); + NetworkTopologyStrategy::DCReplicaCountMap replication_factors; + build_dc_replicas(ks_meta.strategy_options(), &replication_factors); + return SharedRefPtr(new NetworkTopologyStrategy(strategy_class.to_string(), + replication_factors)); } else if (ends_with(strategy_class, SimpleStrategy::STRATEGY_CLASS)) { - return SharedRefPtr( - new SimpleStrategy(strategy_class, ks_meta.strategy_options())); + size_t replication_factor = get_replication_factor(ks_meta.strategy_options()); + return SharedRefPtr(new SimpleStrategy(strategy_class.to_string(), replication_factor)); } else { - return SharedRefPtr(new NonReplicatedStrategy(strategy_class)); + return SharedRefPtr(new NonReplicatedStrategy(strategy_class.to_string())); } } - const std::string NetworkTopologyStrategy::STRATEGY_CLASS("NetworkTopologyStrategy"); -NetworkTopologyStrategy::NetworkTopologyStrategy(const std::string& strategy_class, - const Value* strategy_options) - : ReplicationStrategy(strategy_class) { - build_dc_replicas(strategy_options, &replication_factors_); -} - bool NetworkTopologyStrategy::equal(const KeyspaceMetadata& ks_meta) { - if (strategy_class_ != ks_meta.strategy_class()) return false; + if (ks_meta.strategy_class() != strategy_class_) return false; DCReplicaCountMap temp_rfs; build_dc_replicas(ks_meta.strategy_options(), &temp_rfs); return replication_factors_ == temp_rfs; @@ -135,38 +157,10 @@ void NetworkTopologyStrategy::tokens_to_replicas(const TokenHostMap& primary, To } } -void NetworkTopologyStrategy::build_dc_replicas(const Value* strategy_options, - NetworkTopologyStrategy::DCReplicaCountMap* output) { - if (strategy_options != NULL) { - MapIterator itr(strategy_options); - while (itr.next()) { - StringRef key = itr.key()->to_string_ref(); - StringRef value = itr.value()->to_string_ref(); - if (key != "class") { - size_t replica_count = strtoul(value.to_string().c_str(), NULL, 10); - if (replica_count > 0) { - (*output)[key.to_string()] = replica_count; - } - } - } - } -} - - const std::string SimpleStrategy::STRATEGY_CLASS("SimpleStrategy"); -SimpleStrategy::SimpleStrategy(const std::string& strategy_class, - const Value* strategy_options) - : ReplicationStrategy(strategy_class) - , replication_factor_(0) { - replication_factor_ = get_replication_factor(strategy_options); - if (replication_factor_ == 0) { - LOG_WARN("Replication factor of 0"); - } -} - bool SimpleStrategy::equal(const KeyspaceMetadata& ks_meta) { - if (strategy_class_ != ks_meta.strategy_class()) return false; + if (ks_meta.strategy_class() != strategy_class_) return false; return replication_factor_ == get_replication_factor(ks_meta.strategy_options()); } @@ -187,23 +181,8 @@ void SimpleStrategy::tokens_to_replicas(const TokenHostMap& primary, TokenReplic } } -size_t SimpleStrategy::get_replication_factor(const Value* strategy_options) { - if (strategy_options != NULL) { - MapIterator itr(strategy_options); - while (itr.next()) { - StringRef key = itr.key()->to_string_ref(); - StringRef value = itr.value()->to_string_ref(); - if (key == "replication_factor") { - return strtoul(value.to_string().c_str(), NULL, 10); - } - } - } - return 0; -} - bool NonReplicatedStrategy::equal(const KeyspaceMetadata& ks_meta) { - if (strategy_class_ != ks_meta.strategy_class()) return false; - return true; + return ks_meta.strategy_class() == strategy_class_; } void NonReplicatedStrategy::tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const { diff --git a/src/replication_strategy.hpp b/src/replication_strategy.hpp index c3f5888bc..b43d40c26 100644 --- a/src/replication_strategy.hpp +++ b/src/replication_strategy.hpp @@ -37,9 +37,9 @@ class ReplicationStrategy : public RefCounted { static SharedRefPtr from_keyspace_meta(const KeyspaceMetadata& ks_meta); ReplicationStrategy(const std::string& strategy_class) - : strategy_class_(strategy_class) {} + : strategy_class_(strategy_class) { } - virtual ~ReplicationStrategy() {} + virtual ~ReplicationStrategy() { } virtual bool equal(const KeyspaceMetadata& ks_meta) = 0; virtual void tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const = 0; @@ -55,20 +55,16 @@ class NetworkTopologyStrategy : public ReplicationStrategy { static const std::string STRATEGY_CLASS; NetworkTopologyStrategy(const std::string& strategy_class, - const Value* strategy_options); - virtual ~NetworkTopologyStrategy() {} + const DCReplicaCountMap& replication_factors) + : ReplicationStrategy(strategy_class) + , replication_factors_(replication_factors) { } + + virtual ~NetworkTopologyStrategy() { } virtual bool equal(const KeyspaceMetadata& ks_meta); virtual void tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const; - // Testing only - NetworkTopologyStrategy(const std::string& strategy_class, - const DCReplicaCountMap& replication_factors) - : ReplicationStrategy(strategy_class) - , replication_factors_(replication_factors) {} - private: - static void build_dc_replicas(const Value* strategy_options, DCReplicaCountMap* dc_replicas); DCReplicaCountMap replication_factors_; }; @@ -78,20 +74,16 @@ class SimpleStrategy : public ReplicationStrategy { static const std::string STRATEGY_CLASS; SimpleStrategy(const std::string& strategy_class, - const Value* strategy_options); - virtual ~SimpleStrategy() {} + size_t replication_factor) + : ReplicationStrategy(strategy_class) + , replication_factor_(replication_factor) { } + + virtual ~SimpleStrategy() { } virtual bool equal(const KeyspaceMetadata& ks_meta); virtual void tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const; - // Testing only - SimpleStrategy(const std::string& strategy_class, - size_t replication_factor) - : ReplicationStrategy(strategy_class) - , replication_factor_(replication_factor) {} - private: - static size_t get_replication_factor(const Value* strategy_options); size_t replication_factor_; }; @@ -99,8 +91,8 @@ class SimpleStrategy : public ReplicationStrategy { class NonReplicatedStrategy : public ReplicationStrategy { public: NonReplicatedStrategy(const std::string& strategy_class) - : ReplicationStrategy(strategy_class) {} - virtual ~NonReplicatedStrategy() {} + : ReplicationStrategy(strategy_class) { } + virtual ~NonReplicatedStrategy() { } virtual bool equal(const KeyspaceMetadata& ks_meta); virtual void tokens_to_replicas(const TokenHostMap& primary, TokenReplicaMap* output) const; diff --git a/src/result_metadata.hpp b/src/result_metadata.hpp index c24b4e94e..d7ebea922 100644 --- a/src/result_metadata.hpp +++ b/src/result_metadata.hpp @@ -38,7 +38,7 @@ struct ColumnDefinition : public HashTableEntry { StringRef name; StringRef keyspace; StringRef table; - SharedRefPtr data_type; + DataType::ConstPtr data_type; }; class ResultMetadata : public RefCounted { diff --git a/src/result_response.cpp b/src/result_response.cpp index 8e6c83268..b4e6fd3a0 100644 --- a/src/result_response.cpp +++ b/src/result_response.cpp @@ -277,7 +277,7 @@ char* ResultResponse::decode_metadata(char* input, SharedRefPtr* buffer = decode_string(buffer, &def.name); DataTypeDecoder type_decoder(buffer); - def.data_type = SharedRefPtr(type_decoder.decode()); + def.data_type = DataType::ConstPtr(type_decoder.decode()); buffer = type_decoder.buffer(); (*metadata)->add(def); diff --git a/src/token_map.cpp b/src/token_map.cpp index 3c7b82e76..25b93e4cb 100644 --- a/src/token_map.cpp +++ b/src/token_map.cpp @@ -99,7 +99,6 @@ void TokenMap::clear() { token_map_.clear(); keyspace_replica_map_.clear(); keyspace_strategy_map_.clear(); - partitioner_.reset(); } void TokenMap::build() { diff --git a/src/tuple.hpp b/src/tuple.hpp index 3e628fc35..725da91ed 100644 --- a/src/tuple.hpp +++ b/src/tuple.hpp @@ -40,7 +40,7 @@ class Tuple { : data_type_(new TupleType()) , items_(item_count) { } - explicit Tuple(const SharedRefPtr& data_type) + explicit Tuple(const DataType::ConstPtr& data_type) : data_type_(data_type) , items_(data_type_->types().size()) { } diff --git a/src/user_type_field_iterator.hpp b/src/user_type_field_iterator.hpp index 713c9c814..aed3edba1 100644 --- a/src/user_type_field_iterator.hpp +++ b/src/user_type_field_iterator.hpp @@ -31,7 +31,7 @@ class UserTypeFieldIterator : public Iterator { : Iterator(CASS_ITERATOR_TYPE_USER_TYPE_FIELD) , user_type_value_(user_type_value) , position_(user_type_value->data()) { - SharedRefPtr user_type(user_type_value->data_type()); + UserType::ConstPtr user_type(user_type_value->data_type()); next_ = user_type->fields().begin(); end_ = user_type->fields().end(); } diff --git a/src/user_type_value.hpp b/src/user_type_value.hpp index b8b5df46c..f71f358b1 100644 --- a/src/user_type_value.hpp +++ b/src/user_type_value.hpp @@ -26,23 +26,23 @@ namespace cass { class UserTypeValue : public AbstractData { public: - UserTypeValue(const SharedRefPtr& data_type) + UserTypeValue(const UserType::ConstPtr& data_type) : AbstractData(data_type->fields().size()) , data_type_(data_type) { } - const SharedRefPtr& data_type() const { return data_type_; } + const UserType::ConstPtr& data_type() const { return data_type_; } protected: virtual size_t get_indices(StringRef name, IndexVec* indices) { return data_type_->get_indices(name, indices); } - virtual const SharedRefPtr& get_type(size_t index) const { + virtual const DataType::ConstPtr& get_type(size_t index) const { return data_type_->fields()[index].type; } private: - SharedRefPtr data_type_; + UserType::ConstPtr data_type_; private: DISALLOW_COPY_AND_ASSIGN(UserTypeValue); diff --git a/src/value.cpp b/src/value.cpp index 96c34c1f8..fab9db4a3 100644 --- a/src/value.cpp +++ b/src/value.cpp @@ -183,7 +183,7 @@ CassValueType cass_value_secondary_sub_type(const CassValue* collection) { namespace cass { Value::Value(int protocol_version, - const SharedRefPtr& data_type, + const DataType::ConstPtr& data_type, char* data, int32_t size) : protocol_version_(protocol_version) , data_type_(data_type) { @@ -199,7 +199,7 @@ Value::Value(int protocol_version, SharedRefPtr sub_types(data_type); count_ = sub_types->types().size(); } else if (data_type->is_user_type()) { - SharedRefPtr user_type(data_type); + UserType::ConstPtr user_type(data_type); count_ = user_type->fields().size(); } else { count_ = 0; diff --git a/src/value.hpp b/src/value.hpp index 2ad7bb5a4..52a8cb9e9 100644 --- a/src/value.hpp +++ b/src/value.hpp @@ -31,11 +31,11 @@ class Value { , size_(-1) { } Value(int protocol_version, - const SharedRefPtr& data_type, + const DataType::ConstPtr& data_type, char* data, int32_t size); Value(int protocol_version, - const SharedRefPtr& data_type, + const DataType::ConstPtr& data_type, int32_t count, char* data, int32_t size) : protocol_version_(protocol_version) , data_type_(data_type) @@ -52,23 +52,23 @@ class Value { return data_type_->value_type(); } - const SharedRefPtr& data_type() const { + const DataType::ConstPtr& data_type() const { return data_type_; } CassValueType primary_value_type() const { - const SharedRefPtr& primary(primary_data_type()); + const DataType::ConstPtr& primary(primary_data_type()); if (!primary) { return CASS_VALUE_TYPE_UNKNOWN; } return primary->value_type(); } - const SharedRefPtr& primary_data_type() const { + const DataType::ConstPtr& primary_data_type() const { if (!data_type_ || !data_type_->is_collection()) { return DataType::NIL; } - const SharedRefPtr& collection_type(data_type_); + const CollectionType::ConstPtr& collection_type(data_type_); if (collection_type->types().size() < 1) { return DataType::NIL; } @@ -76,18 +76,18 @@ class Value { } CassValueType secondary_value_type() const { - const SharedRefPtr& secondary(secondary_data_type()); + const DataType::ConstPtr& secondary(secondary_data_type()); if (!secondary) { return CASS_VALUE_TYPE_UNKNOWN; } return secondary->value_type(); } - const SharedRefPtr& secondary_data_type() const { + const DataType::ConstPtr& secondary_data_type() const { if (!data_type_ || !data_type_->is_map()) { return DataType::NIL; } - const SharedRefPtr& collection_type(data_type_); + const CollectionType::ConstPtr& collection_type(data_type_); if (collection_type->types().size() < 2) { return DataType::NIL; } @@ -141,7 +141,7 @@ class Value { private: int protocol_version_; - SharedRefPtr data_type_; + DataType::ConstPtr data_type_; int32_t count_; char* data_; diff --git a/test/ccm_bridge/data/config.txt b/test/ccm_bridge/data/config.txt index 5f955d4ec..daef9f400 100644 --- a/test/ccm_bridge/data/config.txt +++ b/test/ccm_bridge/data/config.txt @@ -13,7 +13,7 @@ # # Uncomment to specify Cassandra version ## -#CASSANDRA_VERSION=2.2.3 +#CASSANDRA_VERSION=3.0.0 ## # Flag to determine if Cassandra version should be obtained from ASF git # diff --git a/test/ccm_bridge/src/bridge.cpp b/test/ccm_bridge/src/bridge.cpp index accaddd6c..61df1136b 100644 --- a/test/ccm_bridge/src/bridge.cpp +++ b/test/ccm_bridge/src/bridge.cpp @@ -1331,6 +1331,11 @@ std::vector CCM::Bridge::generate_create_updateconf_command(CassVer updateconf_command.push_back("enable_user_defined_functions:true"); } + // Create Cassandra version specific updated (C* 3.0+) + if (cassandra_version >= CassVersion("3.0.0")) { + updateconf_command.push_back("enable_scripted_user_defined_functions:true"); + } + return updateconf_command; } diff --git a/test/ccm_bridge/src/bridge.hpp b/test/ccm_bridge/src/bridge.hpp index 4c3d6e009..85a863f8b 100644 --- a/test/ccm_bridge/src/bridge.hpp +++ b/test/ccm_bridge/src/bridge.hpp @@ -39,7 +39,7 @@ typedef struct _LIBSSH2_CHANNEL LIBSSH2_CHANNEL; #endif // Default values -#define DEFAULT_CASSANDRA_VERSION CassVersion("2.2.3") +#define DEFAULT_CASSANDRA_VERSION CassVersion("3.0.0") #define DEFAULT_USE_ASFGIT false #define DEFAULT_CLUSTER_PREFIX "cpp-driver" #define DEFAULT_DEPLOYMENT DeploymentType::LOCAL diff --git a/test/ccm_bridge/src/cass_version.hpp b/test/ccm_bridge/src/cass_version.hpp index 15da610d3..21b2289c3 100644 --- a/test/ccm_bridge/src/cass_version.hpp +++ b/test/ccm_bridge/src/cass_version.hpp @@ -62,7 +62,20 @@ namespace CCM { , patch(0) , extra("") { from_string(version_string); - }; + } + + int compare(const CassVersion& rhs) { + if (major < rhs.major) return -1; + if (major > rhs.major) return 1; + + if (minor < rhs.minor) return -1; + if (minor > rhs.minor) return 1; + + if (patch < rhs.patch) return -1; + if (patch > rhs.patch) return 1; + + return 0; + } /** * Equal comparison operator overload @@ -75,13 +88,7 @@ namespace CCM { * @return True if LHS == RHS; false otherwise */ bool operator ==(const CassVersion& rhs) { - // Check version properties for equality (except extra property) - if (major == rhs.major && - minor == rhs.minor && - patch == rhs.patch) { - return true; - } - return false; + return compare(rhs) == 0; } /** @@ -96,8 +103,7 @@ namespace CCM { */ bool operator ==(const std::string& version) { // Check version properties for equality (except extra property) - CassVersion rhs(version); - return (*this == rhs); + return compare(CassVersion(version)) == 0; } /** @@ -112,7 +118,7 @@ namespace CCM { * @return True if LHS != RHS; false otherwise */ bool operator !=(const CassVersion& rhs) { - return !(*this == rhs); + return compare(rhs) != 0; } /** @@ -127,7 +133,7 @@ namespace CCM { * @return True if LHS != RHS; false otherwise */ bool operator !=(const std::string& version) { - return !(*this == version); + return compare(CassVersion(version)) != 0; } /** @@ -141,17 +147,7 @@ namespace CCM { * @return True if LHS < RHS; false otherwise */ bool operator <(const CassVersion& rhs) { - // Check version properties (except extra property) - if (major < rhs.major) { - return true; - } - if (minor < rhs.minor) { - return true; - } - if (patch < rhs.patch) { - return true; - } - return false; + return compare(rhs) < 0; } /** @@ -165,8 +161,7 @@ namespace CCM { * @return True if LHS < RHS; false otherwise */ bool operator <(const std::string& version) { - CassVersion rhs(version); - return (*this < rhs); + return compare(CassVersion(version)) < 0; } /** @@ -181,7 +176,7 @@ namespace CCM { * @return True if LHS > RHS; false otherwise */ bool operator >(const CassVersion& rhs) { - return (*this > rhs); + return compare(rhs) > 0; } /** @@ -196,8 +191,7 @@ namespace CCM { * @return True if LHS > RHS; false otherwise */ bool operator >(const std::string& version) { - CassVersion rhs(version); - return (*this > rhs); + return compare(CassVersion(version)) > 0; } /** @@ -212,7 +206,7 @@ namespace CCM { * @return True if LHS <= RHS; false otherwise */ bool operator <=(const CassVersion& rhs) { - return !(*this > rhs); + return compare(rhs) <= 0; } /** @@ -227,8 +221,7 @@ namespace CCM { * @return True if LHS <= RHS; false otherwise */ bool operator <=(const std::string& version) { - CassVersion rhs(version); - return !(*this > rhs); + return compare(CassVersion(version)) <= 0; } /** @@ -243,7 +236,7 @@ namespace CCM { * @return True if LHS >= RHS; false otherwise */ bool operator >=(const CassVersion& rhs) { - return !(*this < rhs); + return compare(rhs) >= 0; } /** @@ -258,8 +251,7 @@ namespace CCM { * @return True if LHS >= RHS; false otherwise */ bool operator >=(const std::string& version) { - CassVersion rhs(version); - return !(*this < rhs); + return compare(CassVersion(version)) >= 0; } /** diff --git a/test/integration_tests/src/test_authentication.cpp b/test/integration_tests/src/test_authentication.cpp index 072b489df..d021071c8 100644 --- a/test/integration_tests/src/test_authentication.cpp +++ b/test/integration_tests/src/test_authentication.cpp @@ -17,6 +17,8 @@ #include "cassandra.h" #include "test_utils.hpp" +#include + #include #include #include @@ -28,7 +30,8 @@ struct AthenticationTests { AthenticationTests() : cluster(cass_cluster_new()) - , ccm(new CCM::Bridge("config.txt")) { + , ccm(new CCM::Bridge("config.txt")) + , version(test_utils::get_version()) { ccm->create_cluster(); ccm->kill_cluster(); ccm->update_cluster_configuration("authenticator", "PasswordAuthenticator"); @@ -48,7 +51,9 @@ struct AthenticationTests { test_utils::CassSessionPtr session(test_utils::create_session(cluster.get())); test_utils::CassResultPtr result; - test_utils::execute_query(session.get(), "SELECT * FROM system.schema_keyspaces", &result); + std::stringstream query; + query << "SELECT * FROM " << (version >= "3.0.0" ? "system_schema.keyspaces" : "system.schema_keyspaces"); + test_utils::execute_query(session.get(), query.str().c_str(), &result); BOOST_CHECK(cass_result_row_count(result.get()) > 0); } @@ -69,14 +74,19 @@ struct AthenticationTests { test_utils::CassClusterPtr cluster; boost::shared_ptr ccm; + CCM::CassVersion version; }; BOOST_FIXTURE_TEST_SUITE(authentication, AthenticationTests) BOOST_AUTO_TEST_CASE(protocol_versions) { - auth(1); - auth(2); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + if (version < "2.2.0") { + auth(1); + auth(2); + } auth(3); auth(4); } @@ -88,8 +98,12 @@ BOOST_AUTO_TEST_CASE(empty_credentials) // This test serves to characterize what is there presently. const char* expected_error = "Key may not be empty"; - invalid_credentials(1, "", "", expected_error, CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); - invalid_credentials(2, "", "", expected_error, CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + if (version < "2.2.0") { + invalid_credentials(1, "", "", expected_error, CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); + invalid_credentials(2, "", "", expected_error, CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); + } invalid_credentials(3, "", "", expected_error, CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); invalid_credentials(4, "", "", expected_error, CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); } @@ -98,8 +112,12 @@ BOOST_AUTO_TEST_CASE(bad_credentials) { const char* expected_error = "had the following error on startup: Username and/or password are incorrect"; - invalid_credentials(1, "invalid", "invalid", expected_error, CASS_ERROR_SERVER_BAD_CREDENTIALS); - invalid_credentials(2, "invalid", "invalid", expected_error, CASS_ERROR_SERVER_BAD_CREDENTIALS); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + if (version < "2.2.0") { + invalid_credentials(1, "invalid", "invalid", expected_error, CASS_ERROR_SERVER_BAD_CREDENTIALS); + invalid_credentials(2, "invalid", "invalid", expected_error, CASS_ERROR_SERVER_BAD_CREDENTIALS); + } invalid_credentials(3, "invalid", "invalid", expected_error, CASS_ERROR_SERVER_BAD_CREDENTIALS); invalid_credentials(4, "invalid", "invalid", expected_error, CASS_ERROR_SERVER_BAD_CREDENTIALS); } diff --git a/test/integration_tests/src/test_basics.cpp b/test/integration_tests/src/test_basics.cpp index 9ccfef010..1b956dea6 100644 --- a/test/integration_tests/src/test_basics.cpp +++ b/test/integration_tests/src/test_basics.cpp @@ -670,7 +670,7 @@ BOOST_AUTO_TEST_CASE(unset_parameters) CassError rc = cass_future_error_code(future.get()); - if (version.major >= 2 && version.minor >= 2) { + if (version >= "2.2.0") { // C* 2.2+ uses the value UNSET and that makes this statement a no-op BOOST_REQUIRE(rc == CASS_OK); } else { diff --git a/test/integration_tests/src/test_batch.cpp b/test/integration_tests/src/test_batch.cpp index 62da20ec1..4f9a591c0 100644 --- a/test/integration_tests/src/test_batch.cpp +++ b/test/integration_tests/src/test_batch.cpp @@ -137,7 +137,8 @@ BOOST_AUTO_TEST_CASE(mixed) test_utils::wait_and_check_error(prepared_future.get()); test_utils::CassPreparedPtr prepared(cass_future_get_prepared(prepared_future.get())); - for (int x = 0; x < 1000; x++) + int batch_size = (version >= "2.2.0" ? 100 : 1000); + for (int x = 0; x < batch_size; x++) { test_utils::CassStatementPtr insert_statement; if (x % 2 == 0) { @@ -153,7 +154,7 @@ BOOST_AUTO_TEST_CASE(mixed) test_utils::CassFuturePtr insert_future(cass_session_execute_batch(tester.session, batch.get())); test_utils::wait_and_check_error(insert_future.get()); - tester.validate_results(1000); + tester.validate_results(batch_size); } else { std::cout << "Unsupported Test for Cassandra v" << version.to_string() << ": Skipping batch/mixed" << std::endl; BOOST_REQUIRE(true); @@ -195,7 +196,8 @@ BOOST_AUTO_TEST_CASE(counter_mixed) test_utils::wait_and_check_error(prepared_future.get()); test_utils::CassPreparedPtr prepared(cass_future_get_prepared(prepared_future.get())); - for (int x = 0; x < 1000; x++) + unsigned int batch_size = (version >= "2.2.0" ? 100 : 1000); + for (unsigned int x = 0; x < batch_size; x++) { test_utils::CassStatementPtr update_statement; if (x % 2 == 0) { @@ -218,7 +220,7 @@ BOOST_AUTO_TEST_CASE(counter_mixed) test_utils::execute_query(tester.session, select_query, &result, CASS_CONSISTENCY_QUORUM); - BOOST_REQUIRE(cass_result_row_count(result.get()) == 1000); + BOOST_REQUIRE(cass_result_row_count(result.get()) == batch_size); BOOST_REQUIRE(cass_result_column_count(result.get()) == 2); test_utils::CassIteratorPtr iterator(cass_iterator_from_result(result.get())); diff --git a/test/integration_tests/src/test_consistency.cpp b/test/integration_tests/src/test_consistency.cpp index eb4d87ec6..ba057d522 100644 --- a/test/integration_tests/src/test_consistency.cpp +++ b/test/integration_tests/src/test_consistency.cpp @@ -72,7 +72,13 @@ BOOST_AUTO_TEST_CASE(simple_two_nodes) CassError init_result = policy_tool.init_return_error(session.get(), 12, CASS_CONSISTENCY_EACH_QUORUM); // Should fail (EACH_QUORUM is incompatible with SimpleStrategy) CassError query_result = policy_tool.query_return_error(session.get(), 12, CASS_CONSISTENCY_EACH_QUORUM); // Should fail (see above) BOOST_CHECK_EQUAL(init_result, CASS_OK); // TODO(mpenick): Shouldn't be CASS_OK? - BOOST_CHECK_EQUAL(query_result, CASS_ERROR_SERVER_INVALID_QUERY); + // Handle EACH_QUORUM read support added to C* v3.0.0 [CASSANDRA-9602] + // https://issues.apache.org/jira/browse/CASSANDRA-9602 + if (test_utils::get_version() >= "3.0.0") { + BOOST_CHECK_EQUAL(query_result, CASS_OK); + } else { + BOOST_CHECK_EQUAL(query_result, CASS_ERROR_SERVER_INVALID_QUERY); + } } { CassError init_result = policy_tool.init_return_error(session.get(), 12, CASS_CONSISTENCY_THREE); // Should fail (N=2, RF=1) diff --git a/test/integration_tests/src/test_control_connection.cpp b/test/integration_tests/src/test_control_connection.cpp index fea93baa8..0490fb13d 100644 --- a/test/integration_tests/src/test_control_connection.cpp +++ b/test/integration_tests/src/test_control_connection.cpp @@ -32,18 +32,21 @@ struct ControlConnectionTests { public: boost::shared_ptr ccm; std::string ip_prefix; + CCM::CassVersion version; ControlConnectionTests() : ccm(new CCM::Bridge("config.txt")) - , ip_prefix(ccm->get_ip_prefix()) {} + , ip_prefix(ccm->get_ip_prefix()) + , version(test_utils::get_version()) {} void check_for_live_hosts(test_utils::CassSessionPtr session, const std::set& should_be_present) { std::set hosts; + std::stringstream query; + query << "SELECT * FROM " << (version >= "3.0.0" ? "system_schema.keyspaces" : "system.schema_keyspaces"); for (size_t i = 0; i < should_be_present.size() + 2; ++i) { - const char* query = "SELECT * FROM system.schema_keyspaces"; - test_utils::CassStatementPtr statement(cass_statement_new(query, 0)); + test_utils::CassStatementPtr statement(cass_statement_new(query.str().c_str(), 0)); test_utils::CassFuturePtr future(cass_session_execute(session.get(), statement.get())); if (cass_future_error_code(future.get()) == CASS_OK) { hosts.insert(cass::get_host_from_future(future.get())); @@ -132,7 +135,8 @@ BOOST_AUTO_TEST_CASE(reconnection) ccm->stop_node(1); // Add a new node to make sure the node gets added on the new control connection to node 2 - ccm->bootstrap_node("-Dcassandra.consistent.rangemovement=false"); // Allow this node to come up without node1 + int node = ccm->bootstrap_node("-Dcassandra.consistent.rangemovement=false"); // Allow this node to come up without node1 + test_utils::wait_for_node_connection(ip_prefix, node); // Stop the other node ccm->stop_node(2); @@ -159,7 +163,8 @@ BOOST_AUTO_TEST_CASE(topology_change) test_utils::CassSessionPtr session(test_utils::create_session(cluster.get())); // Adding a new node will trigger a "NEW_NODE" event - ccm->bootstrap_node(); + int node = ccm->bootstrap_node(); + test_utils::wait_for_node_connection(ip_prefix, node); std::set should_be_present = build_ip_range(ip_prefix, 1, 2); check_for_live_hosts(session, should_be_present); @@ -199,7 +204,8 @@ BOOST_AUTO_TEST_CASE(status_change) check_for_live_hosts(session, should_be_present); // Starting a node will trigger an "UP" event - ccm->start_node(2); + int node = ccm->start_node(2); + test_utils::wait_for_node_connection(ip_prefix, node); should_be_present.insert(ip_prefix + "2"); check_for_live_hosts(session, should_be_present); @@ -222,7 +228,7 @@ BOOST_AUTO_TEST_CASE(node_discovery) test_utils::CassSessionPtr session(test_utils::create_session(cluster.get())); // Allow the nodes to be discovered - boost::this_thread::sleep_for(boost::chrono::seconds(10)); //TODO: Remove sleep and implement a pre-check + boost::this_thread::sleep_for(boost::chrono::seconds(20)); //TODO: Remove sleep and implement a pre-check check_for_live_hosts(session, build_ip_range(ip_prefix, 1, 3)); } @@ -251,7 +257,7 @@ BOOST_AUTO_TEST_CASE(node_discovery_invalid_ips) test_utils::CassSessionPtr session(test_utils::create_session(cluster.get(), NULL, 60 * test_utils::ONE_SECOND_IN_MICROS)); // Allow the nodes to be discovered - boost::this_thread::sleep_for(boost::chrono::seconds(10)); //TODO: Remove sleep and implement a pre-check + boost::this_thread::sleep_for(boost::chrono::seconds(20)); //TODO: Remove sleep and implement a pre-check check_for_live_hosts(session, build_ip_range(ip_prefix, 1, 3)); } @@ -281,7 +287,7 @@ BOOST_AUTO_TEST_CASE(node_discovery_no_local_rows) test_utils::CassSessionPtr session(test_utils::create_session(cluster.get())); // Allow the nodes to be discovered - boost::this_thread::sleep_for(boost::chrono::seconds(10)); //TODO: Remove sleep and implement a pre-check + boost::this_thread::sleep_for(boost::chrono::seconds(20)); //TODO: Remove sleep and implement a pre-check check_for_live_hosts(session, build_ip_range(ip_prefix, 1, 3)); } @@ -341,9 +347,7 @@ BOOST_AUTO_TEST_CASE(full_outage) BOOST_CHECK(test_utils::execute_query_with_error(session.get(), query) == CASS_ERROR_LIB_NO_HOSTS_AVAILABLE); ccm->start_cluster(); - - // Allow the nodes to be discovered - boost::this_thread::sleep_for(boost::chrono::seconds(10)); //TODO: Remove sleep and implement a pre-check + test_utils::wait_for_node_connection(ip_prefix, 1); test_utils::execute_query(session.get(), query); } diff --git a/test/integration_tests/src/test_future_callbacks.cpp b/test/integration_tests/src/test_future_callbacks.cpp index 0eae6163e..a42036d8b 100644 --- a/test/integration_tests/src/test_future_callbacks.cpp +++ b/test/integration_tests/src/test_future_callbacks.cpp @@ -27,6 +27,8 @@ #include "cassandra.h" #include "test_utils.hpp" +#include + namespace { struct CallbackData { @@ -123,7 +125,9 @@ BOOST_AUTO_TEST_CASE(result) test_utils::CassResultPtr result; - test_utils::CassStatementPtr statement(cass_statement_new("SELECT * FROM system.schema_keyspaces", 0)); + std::stringstream query; + query << "SELECT * FROM " << (version >= "3.0.0" ? "system_schema.keyspaces" : "system.schema_keyspaces"); + test_utils::CassStatementPtr statement(cass_statement_new(query.str().c_str(), 0)); test_utils::CassFuturePtr future(cass_session_execute(session.get(), statement.get())); cass_future_set_callback(future.get(), check_result_callback, callback_data.get()); diff --git a/test/integration_tests/src/test_latency_aware_policy.cpp b/test/integration_tests/src/test_latency_aware_policy.cpp index 063bd5239..dc84eb42b 100644 --- a/test/integration_tests/src/test_latency_aware_policy.cpp +++ b/test/integration_tests/src/test_latency_aware_policy.cpp @@ -50,7 +50,13 @@ struct LatencyAwarePolicyTest { test_utils::initialize_contact_points(cluster_.get(), ccm_->get_ip_prefix(), 3, 0); cass_cluster_set_latency_aware_routing(cluster_.get(), cass_true); cass_cluster_set_latency_aware_routing_settings(cluster_.get(), 1e6, 1, 1, 1, 1); - cass_cluster_set_protocol_version(cluster_.get(), 1); // Protocol for this test doesn't matter so simply support all C* versions + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + int protocol_version = 1; + if (test_utils::get_version() >= "3.0.0") { + protocol_version = 3; + } + cass_cluster_set_protocol_version(cluster_.get(), protocol_version); // Protocol for this test doesn't matter so simply support all C* versions // Connect to the cluster session_ = test_utils::create_session(cluster_.get()); diff --git a/test/integration_tests/src/test_prepared_outage.cpp b/test/integration_tests/src/test_prepared_outage.cpp index 2813a1022..f019d33f1 100644 --- a/test/integration_tests/src/test_prepared_outage.cpp +++ b/test/integration_tests/src/test_prepared_outage.cpp @@ -126,7 +126,6 @@ BOOST_AUTO_TEST_CASE(reprepared_on_new_node) test_utils::execute_query(session, str(boost::format(insert_query) % table_name % "123456789" % 20)); ccm->enable_node_gossip(2); - for (int i = 0; i < 10; ++i) { test_utils::CassStatementPtr statement(cass_prepared_bind(prepared.get())); BOOST_REQUIRE(cass_statement_set_consistency(statement.get(), CASS_CONSISTENCY_QUORUM) == CASS_OK); diff --git a/test/integration_tests/src/test_schema_metadata.cpp b/test/integration_tests/src/test_schema_metadata.cpp index 1cfa66d9b..5186facb7 100644 --- a/test/integration_tests/src/test_schema_metadata.cpp +++ b/test/integration_tests/src/test_schema_metadata.cpp @@ -66,9 +66,10 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { test_utils::CassResultPtr result; for (int i = 0; i < 10; ++i) { + const char* system_schema_ks = version >= "3.0.0" ? "system_schema.keyspaces" : "system.schema_keyspaces"; test_utils::execute_query(session, str(boost::format( - "SELECT * FROM system.schema_keyspaces WHERE keyspace_name = '%s'") % ks), &result); + "SELECT * FROM %s WHERE keyspace_name = '%s'") % system_schema_ks % ks), &result); if (cass_result_row_count(result.get()) > 0) { return; } @@ -85,7 +86,7 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { for (size_t i = 0; i < 10 && cass_schema_meta_snapshot_version(schema_meta_) == cass_schema_meta_snapshot_version(old); ++i) { - boost::this_thread::sleep_for(boost::chrono::milliseconds(10)); + boost::this_thread::sleep_for(boost::chrono::milliseconds(1000)); cass_schema_meta_free(schema_meta_); schema_meta_ = cass_session_get_schema_meta(session); } @@ -182,7 +183,6 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { cass_iterator_get_meta_field_name(itr.get(), &name.data, &name.length); observed.insert(std::string(name.data, name.length)); } - BOOST_REQUIRE_EQUAL_COLLECTIONS(observed.begin(), observed.end(), expected_fields.begin(), expected_fields.end()); } @@ -224,17 +224,28 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { const std::set& column_fields() { static std::set fields; if (fields.empty()) { - fields.insert("keyspace_name"); - fields.insert("columnfamily_name"); - fields.insert("column_name"); - fields.insert("component_index"); - fields.insert("index_name"); - fields.insert("index_options"); - fields.insert("index_type"); - fields.insert("validator"); - - if (version >= "2.0.0") { + if (version >= "3.0.0") { + fields.insert("keyspace_name"); + fields.insert("table_name"); + fields.insert("column_name"); + fields.insert("clustering_order"); + fields.insert("column_name_bytes"); + fields.insert("kind"); + fields.insert("position"); fields.insert("type"); + } else { + fields.insert("keyspace_name"); + fields.insert("columnfamily_name"); + fields.insert("column_name"); + fields.insert("component_index"); + fields.insert("index_name"); + fields.insert("index_options"); + fields.insert("index_type"); + fields.insert("validator"); + + if (version >= "2.0.0") { + fields.insert("type"); + } } } return fields; @@ -278,55 +289,77 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { const std::set& table_fields() { static std::set fields; if (fields.empty()) { - fields.insert("keyspace_name"); - fields.insert("columnfamily_name"); - fields.insert("bloom_filter_fp_chance"); - fields.insert("caching"); - fields.insert("column_aliases"); - fields.insert("comment"); - fields.insert("compaction_strategy_class"); - fields.insert("compaction_strategy_options"); - fields.insert("comparator"); - fields.insert("compression_parameters"); - fields.insert("default_validator"); - fields.insert("gc_grace_seconds"); - fields.insert("id"); - fields.insert("key_alias"); - fields.insert("key_aliases"); - fields.insert("key_validator"); - fields.insert("local_read_repair_chance"); - fields.insert("max_compaction_threshold"); - fields.insert("min_compaction_threshold"); - fields.insert("populate_io_cache_on_flush"); - fields.insert("read_repair_chance"); - fields.insert("replicate_on_write"); - fields.insert("subcomparator"); - fields.insert("type"); - fields.insert("value_alias"); - - if (version >= "2.0.0") { + if (version >= "3.0.0") { + fields.insert("keyspace_name"); + fields.insert("table_name"); + fields.insert("bloom_filter_fp_chance"); + fields.insert("caching"); + fields.insert("comment"); + fields.insert("compaction"); + fields.insert("compression"); + fields.insert("crc_check_chance"); + fields.insert("dclocal_read_repair_chance"); fields.insert("default_time_to_live"); - fields.insert("dropped_columns"); - fields.erase("id"); - fields.insert("index_interval"); - fields.insert("is_dense"); - fields.erase("key_alias"); + fields.insert("extensions"); + fields.insert("flags"); + fields.insert("gc_grace_seconds"); + fields.insert("id"); + fields.insert("max_index_interval"); fields.insert("memtable_flush_period_in_ms"); + fields.insert("min_index_interval"); + fields.insert("read_repair_chance"); fields.insert("speculative_retry"); - - if (version >= "2.1.0") { - fields.insert("cf_id"); - fields.insert("max_index_interval"); - fields.insert("min_index_interval"); - fields.erase("populate_io_cache_on_flush"); - fields.erase("replicate_on_write"); - } - - if (version >= "2.2.0") { - fields.erase("column_aliases"); - fields.erase("key_aliases"); - fields.erase("value_alias"); - fields.erase("index_interval"); + } else { + fields.insert("keyspace_name"); + fields.insert("columnfamily_name"); + fields.insert("bloom_filter_fp_chance"); + fields.insert("caching"); + fields.insert("column_aliases"); + fields.insert("comment"); + fields.insert("compaction_strategy_class"); + fields.insert("compaction_strategy_options"); + fields.insert("comparator"); + fields.insert("compression_parameters"); + fields.insert("default_validator"); + fields.insert("gc_grace_seconds"); + fields.insert("id"); + fields.insert("key_alias"); + fields.insert("key_aliases"); + fields.insert("key_validator"); + fields.insert("local_read_repair_chance"); + fields.insert("max_compaction_threshold"); + fields.insert("min_compaction_threshold"); + fields.insert("populate_io_cache_on_flush"); + fields.insert("read_repair_chance"); + fields.insert("replicate_on_write"); + fields.insert("subcomparator"); + fields.insert("type"); + fields.insert("value_alias"); + + if (version >= "2.0.0") { + fields.insert("default_time_to_live"); + fields.insert("dropped_columns"); + fields.erase("id"); + fields.insert("index_interval"); + fields.insert("is_dense"); + fields.erase("key_alias"); + fields.insert("memtable_flush_period_in_ms"); + fields.insert("speculative_retry"); + + if (version >= "2.1.0") { + fields.insert("cf_id"); + fields.insert("max_index_interval"); + fields.insert("min_index_interval"); + fields.erase("populate_io_cache_on_flush"); + fields.erase("replicate_on_write"); + } + + if (version >= "2.2.0") { + fields.erase("column_aliases"); + fields.erase("key_aliases"); + fields.erase("value_alias"); + fields.erase("index_interval"); + } } } } @@ -340,16 +373,18 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { verify_fields(test_utils::CassIteratorPtr(cass_iterator_fields_from_table_meta(table_meta)), table_fields()); verify_fields_by_name(table_meta, table_fields()); verify_value(cass_table_meta_field_by_name(table_meta, "keyspace_name"), SIMPLE_STRATEGY_KEYSPACE_NAME); - verify_value(cass_table_meta_field_by_name(table_meta, "columnfamily_name"), ALL_DATA_TYPES_TABLE_NAME); + verify_value(cass_table_meta_field_by_name(table_meta, version >= "3.0.0" ? "table_name" : "columnfamily_name"), ALL_DATA_TYPES_TABLE_NAME); // not going for every field, just making sure one of each type (fixed, list, map) is correctly added verify_value(cass_table_meta_field_by_name(table_meta, "comment"), COMMENT); - const CassValue* value = cass_table_meta_field_by_name(table_meta, "compression_parameters"); + + const CassValue* value = cass_table_meta_field_by_name(table_meta, version >= "3.0.0" ? "compression" : "compression_parameters"); + BOOST_REQUIRE(value); BOOST_REQUIRE_EQUAL(cass_value_type(value), CASS_VALUE_TYPE_MAP); BOOST_REQUIRE_GE(cass_value_item_count(value), 1ul); test_utils::CassIteratorPtr itr(cass_iterator_from_map(value)); - const std::string parameter = "sstable_compression"; + const std::string parameter = version >= "3.0.0." ? "class" : "sstable_compression"; bool param_found = false; while (cass_iterator_next(itr.get())) { value = cass_iterator_get_map_key(itr.get()); @@ -363,7 +398,10 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { } BOOST_CHECK(param_found); - if (version >= "2.1.0") { + if (version >= "3.0.0") { + value = cass_table_meta_field_by_name(table_meta, "id"); + BOOST_REQUIRE_EQUAL(cass_value_type(value), CASS_VALUE_TYPE_UUID); + } else if (version >= "2.1.0") { value = cass_table_meta_field_by_name(table_meta, "cf_id"); BOOST_REQUIRE_EQUAL(cass_value_type(value), CASS_VALUE_TYPE_UUID); } else { @@ -406,8 +444,12 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { if (fields.empty()) { fields.insert("keyspace_name"); fields.insert("durable_writes"); - fields.insert("strategy_class"); - fields.insert("strategy_options"); + if (version >= "3.0.0") { + fields.insert("replication"); + } else { + fields.insert("strategy_class"); + fields.insert("strategy_options"); + } } return fields; } @@ -420,9 +462,15 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { verify_fields(test_utils::CassIteratorPtr(cass_iterator_fields_from_keyspace_meta(ks_meta)), keyspace_fields()); verify_fields_by_name(ks_meta, keyspace_fields()); verify_value(cass_keyspace_meta_field_by_name(ks_meta, "keyspace_name"), name); - verify_value(cass_keyspace_meta_field_by_name(ks_meta, "durable_writes"), (cass_bool_t)durable_writes); - verify_value(cass_keyspace_meta_field_by_name(ks_meta, "strategy_class"), strategy_class); - verify_value(cass_keyspace_meta_field_by_name(ks_meta, "strategy_options"), strategy_options); + //verify_value(cass_keyspace_meta_field_by_name(ks_meta, "durable_writes"), (cass_bool_t)durable_writes); + if (version >= "3.0.0") { + std::map replication(strategy_options); + replication["class"] = strategy_class; + verify_value(cass_keyspace_meta_field_by_name(ks_meta, "replication"), replication); + } else { + verify_value(cass_keyspace_meta_field_by_name(ks_meta, "strategy_class"), strategy_class); + verify_value(cass_keyspace_meta_field_by_name(ks_meta, "strategy_options"), strategy_options); + } BOOST_CHECK(!cass_keyspace_meta_table_by_name(ks_meta, "some bogus entry")); } @@ -441,7 +489,9 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { size_t keyspace_count = 0; while (cass_iterator_next(itr.get())) ++keyspace_count; size_t number_of_default_keyspaces = 2; - if (version >= "2.2.0") { + if (version >= "3.0.0") { + number_of_default_keyspaces = 5; + } else if (version >= "2.2.0") { number_of_default_keyspaces = 4; } BOOST_CHECK_EQUAL(keyspace_count, number_of_default_keyspaces); @@ -640,12 +690,23 @@ struct TestSchemaMetadata : public test_utils::SingleSessionTest { cass_function_meta_name(func_meta, &agg_meta_string_value.data, &agg_meta_string_value.length); BOOST_CHECK(test_utils::Value::equal(CassString(udf_final_name.c_str()), agg_meta_string_value)); - // Aggregate initial condition (type and value check) const CassValue* agg_init_cond = cass_aggregate_meta_init_cond(agg_meta); - T agg_init_cond_value; - BOOST_CHECK(cass_value_type(agg_init_cond) == return_value_type); - BOOST_CHECK(test_utils::Value::get(agg_init_cond, &agg_init_cond_value) == CASS_OK); - BOOST_REQUIRE(test_utils::Value::equal(init_cond_value, agg_init_cond_value)); + BOOST_REQUIRE(agg_init_cond); + // Aggregate initial condition (type and value check) + if (version >= "3.0.0") { + std::stringstream ss; + ss << init_cond_value; + std::string s(ss.str()); + CassString agg_init_cond_value; + BOOST_CHECK(cass_value_type(agg_init_cond) == CASS_VALUE_TYPE_VARCHAR); + BOOST_CHECK(test_utils::Value::get(agg_init_cond, &agg_init_cond_value) == CASS_OK); + BOOST_REQUIRE(test_utils::Value::equal(CassString(s.data(), s.length()), agg_init_cond_value)); + } else { + T agg_init_cond_value; + BOOST_CHECK(cass_value_type(agg_init_cond) == return_value_type); + BOOST_CHECK(test_utils::Value::get(agg_init_cond, &agg_init_cond_value) == CASS_OK); + BOOST_REQUIRE(test_utils::Value::equal(init_cond_value, agg_init_cond_value)); + } } void verify_user_data_type() { @@ -775,6 +836,7 @@ BOOST_AUTO_TEST_CASE(simple) { BOOST_AUTO_TEST_CASE(keys) { test_utils::execute_query(session, "CREATE KEYSPACE keys WITH replication = " "{ 'class' : 'SimpleStrategy', 'replication_factor' : 3 }"); + refresh_schema_meta(); { test_utils::execute_query(session, "CREATE TABLE keys.single_parition_key (key text, value text, PRIMARY KEY(key))"); @@ -782,7 +844,7 @@ BOOST_AUTO_TEST_CASE(keys) { const CassTableMeta* table_meta = schema_get_table("keys", "single_parition_key"); - BOOST_REQUIRE(cass_table_meta_partition_key_count(table_meta) == 1); + BOOST_REQUIRE_EQUAL(cass_table_meta_partition_key_count(table_meta), 1); verify_partition_key(table_meta, 0, "key"); verify_column_order(table_meta, 1, 0, 2); @@ -795,7 +857,7 @@ BOOST_AUTO_TEST_CASE(keys) { const CassTableMeta* table_meta = schema_get_table("keys", "composite_parition_key"); - BOOST_REQUIRE(cass_table_meta_partition_key_count(table_meta) == 2); + BOOST_REQUIRE_EQUAL(cass_table_meta_partition_key_count(table_meta), 2); verify_partition_key(table_meta, 0, "key1"); verify_partition_key(table_meta, 1, "key2"); @@ -809,10 +871,10 @@ BOOST_AUTO_TEST_CASE(keys) { const CassTableMeta* table_meta = schema_get_table("keys", "composite_key"); - BOOST_REQUIRE(cass_table_meta_partition_key_count(table_meta) == 1); + BOOST_REQUIRE_EQUAL(cass_table_meta_partition_key_count(table_meta), 1); verify_partition_key(table_meta, 0, "key1"); - BOOST_REQUIRE(cass_table_meta_clustering_key_count(table_meta) == 1); + BOOST_REQUIRE_EQUAL(cass_table_meta_clustering_key_count(table_meta), 1); verify_clustering_key(table_meta, 0, "key2"); verify_column_order(table_meta, 1, 1, 3); @@ -825,10 +887,10 @@ BOOST_AUTO_TEST_CASE(keys) { const CassTableMeta* table_meta = schema_get_table("keys", "composite_clustering_key"); - BOOST_REQUIRE(cass_table_meta_partition_key_count(table_meta) == 1); + BOOST_REQUIRE_EQUAL(cass_table_meta_partition_key_count(table_meta), 1); verify_partition_key(table_meta, 0, "key1"); - BOOST_REQUIRE(cass_table_meta_clustering_key_count(table_meta) == 2); + BOOST_REQUIRE_EQUAL(cass_table_meta_clustering_key_count(table_meta), 2); verify_clustering_key(table_meta, 0, "key2"); verify_clustering_key(table_meta, 1, "key3"); @@ -842,11 +904,11 @@ BOOST_AUTO_TEST_CASE(keys) { const CassTableMeta* table_meta = schema_get_table("keys", "composite_parition_and_clustering_key"); - BOOST_REQUIRE(cass_table_meta_partition_key_count(table_meta) == 2); + BOOST_REQUIRE_EQUAL(cass_table_meta_partition_key_count(table_meta), 2); verify_partition_key(table_meta, 0, "key1"); verify_partition_key(table_meta, 1, "key2"); - BOOST_REQUIRE(cass_table_meta_clustering_key_count(table_meta) == 2); + BOOST_REQUIRE_EQUAL(cass_table_meta_clustering_key_count(table_meta), 2); verify_clustering_key(table_meta, 0, "key3"); verify_clustering_key(table_meta, 1, "key4"); @@ -874,8 +936,10 @@ BOOST_AUTO_TEST_CASE(disable) { // Verify schema change event { + refresh_schema_meta(); test_utils::execute_query(session, "CREATE KEYSPACE ks1 WITH replication = " "{ 'class' : 'SimpleStrategy', 'replication_factor' : 3 }"); + refresh_schema_meta(); verify_keyspace_created("ks1"); test_utils::CassSchemaMetaPtr schema_meta(cass_session_get_schema_meta(session)); BOOST_CHECK(cass_schema_meta_keyspace_by_name(schema_meta.get(), "ks1") != NULL); diff --git a/test/integration_tests/src/test_sessions.cpp b/test/integration_tests/src/test_sessions.cpp index 99518eae7..ee7c9ceda 100644 --- a/test/integration_tests/src/test_sessions.cpp +++ b/test/integration_tests/src/test_sessions.cpp @@ -37,6 +37,7 @@ #define SESSION_STRESS_NUMBER_OF_SHARED_SESSION_THREADS 8 //NOTE: Total threads will be (SESSION_STRESS_NUMBER_OF_SESSIONS / 4) * SESSION_STRESS_NUMBER_OF_SHARED_SESSION_THREADS #define SESSION_STRESS_CHAOS_NUMBER_OF_ITERATIONS 256 #define SESSION_STRESS_NUMBER_OF_ITERATIONS 4 //NOTE: This effects sleep timer as well for async log messages +#define SESSION_STRESS_NUMBER_OF_ALLOWED_NO_HOST_AVAILABLE_OCCURRENCES 2 struct SessionTests { public: @@ -504,13 +505,23 @@ void query_sessions(const SessionContainer& sessions) { uv_thread_join(&threads[n]); } + int no_host_count = 0; for (unsigned int n = 0; n < thread_count; ++n) { //Timeouts are OK (especially on the minor chaos test) CassError error_code = queries[n].error_code; if (error_code != CASS_OK && error_code != CASS_ERROR_LIB_REQUEST_TIMED_OUT) { - BOOST_FAIL("Error occurred during query '" << std::string(cass_error_desc(error_code))); + if (error_code == CASS_ERROR_LIB_NO_HOSTS_AVAILABLE) { + ++no_host_count; + } else { + BOOST_FAIL("Error occurred during query '" << std::string(cass_error_desc(error_code)) << "' [" << error_code << "]"); + } } } + + // Ensure that some hosts were available (chaos) + if (no_host_count > SESSION_STRESS_NUMBER_OF_ALLOWED_NO_HOST_AVAILABLE_OCCURRENCES) { + BOOST_FAIL("Unacceptable Limit of CASS_ERROR_LIB_NO_HOSTS_AVAILABLE Occurred: " << no_host_count << " > " << SESSION_STRESS_NUMBER_OF_ALLOWED_NO_HOST_AVAILABLE_OCCURRENCES); + } } /** @@ -605,7 +616,7 @@ BOOST_AUTO_TEST_CASE(stress) BOOST_CHECK_EQUAL(test_utils::CassLog::message_count(), quarter_sessions); BOOST_CHECK_EQUAL(sessions.count(), quarter_sessions); - //Query sessions over multiple threads + //Query sessions over multiple threads query_sessions(sessions); test_utils::CassLog::reset(SESSION_STRESS_CLOSED_LOG_MESSAGE); diff --git a/test/integration_tests/src/test_utils.cpp b/test/integration_tests/src/test_utils.cpp index f0ad19f13..7e6279a01 100644 --- a/test/integration_tests/src/test_utils.cpp +++ b/test/integration_tests/src/test_utils.cpp @@ -108,9 +108,11 @@ void CassLog::callback(const CassLogMessage* message, void* data) { message->message); } boost::lock_guard l(*log_data); - if (log_data->message.empty()) return; - if (str.find(log_data->message) != std::string::npos) { - log_data->message_count++; + if (log_data->messages.empty()) return; + for (std::vector::const_iterator iterator = log_data->messages.begin(); iterator != log_data->messages.end(); ++iterator) { + if (str.find(*iterator) != std::string::npos) { + log_data->message_count++; + } } } @@ -420,5 +422,30 @@ std::string implode(const std::vector& elements, const char delimit return result; } +void wait_for_node_connection(const std::string& ip_prefix, int node, int total_attempts /*= 10*/) { + std::vector nodes; + nodes.push_back(node); + wait_for_node_connections(ip_prefix, nodes, total_attempts); +} + +void wait_for_node_connections(const std::string& ip_prefix, std::vector nodes, int total_attempts /*= 10*/) { + // Build the log messages to look for + for (std::vector::const_iterator iterator = nodes.begin(); iterator != nodes.end(); ++iterator) { + std::stringstream log_message; + log_message << "Connected to host " << ip_prefix << *iterator; + if (iterator == nodes.begin()) { + test_utils::CassLog::reset(log_message.str().c_str()); + } else { + test_utils::CassLog::add(log_message.str().c_str()); + } + } + + int num_of_attempts = 0; + while (num_of_attempts < total_attempts && test_utils::CassLog::message_count() < nodes.size()) { + boost::this_thread::sleep_for(boost::chrono::seconds(1)); + ++num_of_attempts; + } +} + //----------------------------------------------------------------------------------- } // End of namespace test_utils diff --git a/test/integration_tests/src/test_utils.hpp b/test/integration_tests/src/test_utils.hpp index d0a3045ed..239d7465a 100644 --- a/test/integration_tests/src/test_utils.hpp +++ b/test/integration_tests/src/test_utils.hpp @@ -138,6 +138,10 @@ class CassLog{ log_data_.reset(msg); } + static void add(const std::string& msg) { + log_data_.add(msg); + } + static size_t message_count(); static void set_output_log_level(CassLogLevel log_level) { @@ -152,12 +156,17 @@ class CassLog{ void reset(const std::string& msg) { boost::lock_guard l(*this); - message = msg; + messages.clear(); + add(msg); message_count = 0; } + void add(const std::string& msg) { + messages.push_back(msg); + } + boost::mutex m; - std::string message; + std::vector messages; size_t message_count; CassLogLevel output_log_level; }; @@ -1230,6 +1239,25 @@ std::string load_ssl_certificate(const std::string filename); std::string implode(const std::vector& elements, const char delimiter = ' ', const char* delimiter_prefix = NULL, const char* delimiter_suffix = NULL); +/** + * Wait for the driver to establish connection to a given node + * + * @param ip_prefix IPv4 prefix for node + * @param node Node to wait for + * @param total_attempts Total number of attempts to wait on connection + */ +void wait_for_node_connection(const std::string& ip_prefix, int node, int total_attempts = 10); + +/** + * Wait for the driver to establish connection to a given set of nodes + * + * @param ip_prefix IPv4 prefix for node(s) + * @param nodes List of nodes to wait for + * @param total_attempts Total number of attempts to wait on connection + * (default: 10) + */ +void wait_for_node_connections(const std::string& ip_prefix, std::vector nodes, int total_attempts = 10); + extern const char* CREATE_TABLE_ALL_TYPES; extern const char* CREATE_TABLE_ALL_TYPES_V4; extern const char* CREATE_TABLE_TIME_SERIES; diff --git a/test/integration_tests/src/test_version1.cpp b/test/integration_tests/src/test_version1.cpp index 58000070c..198e0af7f 100644 --- a/test/integration_tests/src/test_version1.cpp +++ b/test/integration_tests/src/test_version1.cpp @@ -93,90 +93,118 @@ struct Version1Tests : public test_utils::SingleSessionTest { } }; -BOOST_FIXTURE_TEST_SUITE(version1, Version1Tests) +BOOST_AUTO_TEST_SUITE(version1) BOOST_AUTO_TEST_CASE(query) { - test_utils::execute_query(session, "CREATE TABLE test (key int PRIMARY KEY, v1 int, v2 text, v3 list, v4 set);"); - test_utils::execute_query(session, "INSERT INTO test (key, v1, v2, v3, v4) VALUES (0, 99, 'abc', [ 0, 1, 2 ], { 'd', 'e', 'f' });"); - check_result(session); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + CCM::CassVersion version = test_utils::get_version(); + if (version < "3.0.0") { + Version1Tests tester; + test_utils::execute_query(tester.session, "CREATE TABLE test (key int PRIMARY KEY, v1 int, v2 text, v3 list, v4 set);"); + test_utils::execute_query(tester.session, "INSERT INTO test (key, v1, v2, v3, v4) VALUES (0, 99, 'abc', [ 0, 1, 2 ], { 'd', 'e', 'f' });"); + check_result(tester.session); + } else { + std::cout << "Invalid Test for Cassandra v" << version.to_string() << ": Protocol v1 and v2 were dropped in v3.0.0" << std::endl; + BOOST_REQUIRE(true); + } } BOOST_AUTO_TEST_CASE(prepared) { - test_utils::execute_query(session, "CREATE TABLE test (key int PRIMARY KEY, v1 int, v2 text, v3 list, v4 set);"); - - test_utils::CassPreparedPtr prepared - ( - prepare_statement(session, "INSERT INTO test (key, v1, v2, v3, v4) VALUES (?, ?, ?, ?, ?)")); - - test_utils::CassStatementPtr statement(cass_prepared_bind(prepared.get())); - - cass_statement_bind_int32(statement.get(), 0, 0); - cass_statement_bind_int32(statement.get(), 1, 99); - cass_statement_bind_string(statement.get(), 2, "abc"); - - test_utils::CassCollectionPtr list(cass_collection_new(CASS_COLLECTION_TYPE_LIST, 3)); - cass_collection_append_int32(list.get(), 0); - cass_collection_append_int32(list.get(), 1); - cass_collection_append_int32(list.get(), 2); - cass_statement_bind_collection(statement.get(), 3, list.get()); - - test_utils::CassCollectionPtr set(cass_collection_new(CASS_COLLECTION_TYPE_SET, 3)); - cass_collection_append_string(set.get(), "d"); - cass_collection_append_string(set.get(), "e"); - cass_collection_append_string(set.get(), "f"); - cass_statement_bind_collection(statement.get(), 4, set.get()); - - test_utils::CassFuturePtr future(cass_session_execute(session, statement.get())); - test_utils::wait_and_check_error(future.get()); - - check_result(session); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + CCM::CassVersion version = test_utils::get_version(); + if (version < "3.0.0") { + Version1Tests tester; + test_utils::execute_query(tester.session, "CREATE TABLE test (key int PRIMARY KEY, v1 int, v2 text, v3 list, v4 set);"); + test_utils::CassPreparedPtr prepared(prepare_statement(tester.session, "INSERT INTO test (key, v1, v2, v3, v4) VALUES (?, ?, ?, ?, ?)")); + test_utils::CassStatementPtr statement(cass_prepared_bind(prepared.get())); + + cass_statement_bind_int32(statement.get(), 0, 0); + cass_statement_bind_int32(statement.get(), 1, 99); + cass_statement_bind_string(statement.get(), 2, "abc"); + + test_utils::CassCollectionPtr list(cass_collection_new(CASS_COLLECTION_TYPE_LIST, 3)); + cass_collection_append_int32(list.get(), 0); + cass_collection_append_int32(list.get(), 1); + cass_collection_append_int32(list.get(), 2); + cass_statement_bind_collection(statement.get(), 3, list.get()); + + test_utils::CassCollectionPtr set(cass_collection_new(CASS_COLLECTION_TYPE_SET, 3)); + cass_collection_append_string(set.get(), "d"); + cass_collection_append_string(set.get(), "e"); + cass_collection_append_string(set.get(), "f"); + cass_statement_bind_collection(statement.get(), 4, set.get()); + + test_utils::CassFuturePtr future(cass_session_execute(tester.session, statement.get())); + test_utils::wait_and_check_error(future.get()); + + check_result(tester.session); + } else { + std::cout << "Invalid Test for Cassandra v" << version.to_string() << ": Protocol v1 and v2 were dropped in v3.0.0" << std::endl; + BOOST_REQUIRE(true); + } } BOOST_AUTO_TEST_CASE(batch_error) { - test_utils::execute_query(session, "CREATE TABLE test (key int PRIMARY KEY, value int);"); - - test_utils::CassBatchPtr batch(cass_batch_new(CASS_BATCH_TYPE_LOGGED)); - - for (int x = 0; x < 4; x++) - { - std::string insert_query = str(boost::format("INSERT INTO test (key, value) VALUES(%d, %d);") % x % x); - test_utils::CassStatementPtr insert_statement(cass_statement_new(insert_query.c_str(), 0)); - cass_batch_add_statement(batch.get(), insert_statement.get()); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + CCM::CassVersion version = test_utils::get_version(); + if (version < "3.0.0") { + Version1Tests tester; + test_utils::execute_query(tester.session, "CREATE TABLE test (key int PRIMARY KEY, value int);"); + test_utils::CassBatchPtr batch(cass_batch_new(CASS_BATCH_TYPE_LOGGED)); + + for (int x = 0; x < 4; x++) { + std::string insert_query = str(boost::format("INSERT INTO test (key, value) VALUES(%d, %d);") % x % x); + test_utils::CassStatementPtr insert_statement(cass_statement_new(insert_query.c_str(), 0)); + cass_batch_add_statement(batch.get(), insert_statement.get()); + } + + test_utils::CassFuturePtr future(cass_session_execute_batch(tester.session, batch.get())); + + CassError code = cass_future_error_code(future.get()); + CassString message; + cass_future_error_message(future.get(), &message.data, &message.length); + BOOST_REQUIRE(code == CASS_ERROR_LIB_MESSAGE_ENCODE); + BOOST_REQUIRE(std::string(message.data, message.length).find("Operation unsupported by this protocol version") != std::string::npos); + } else { + std::cout << "Invalid Test for Cassandra v" << version.to_string() << ": Protocol v1 and v2 were dropped in v3.0.0" << std::endl; + BOOST_REQUIRE(true); } - - test_utils::CassFuturePtr future(cass_session_execute_batch(session, batch.get())); - - CassError code = cass_future_error_code(future.get()); - CassString message; - cass_future_error_message(future.get(), &message.data, &message.length); - BOOST_REQUIRE(code == CASS_ERROR_LIB_MESSAGE_ENCODE); - BOOST_REQUIRE(std::string(message.data, message.length).find("Operation unsupported by this protocol version") != std::string::npos); } BOOST_AUTO_TEST_CASE(query_param_error) { - test_utils::execute_query(session, "CREATE TABLE test (key int PRIMARY KEY, value int);"); - - const char* insert_query = "INSERT INTO test (key, value) VALUES(?, ?);"; - - test_utils::CassStatementPtr statement(cass_statement_new(insert_query, 2)); - - cass_statement_bind_int32(statement.get(), 0, 11); - cass_statement_bind_int32(statement.get(), 1, 99); - - test_utils::CassFuturePtr future(cass_session_execute(session, statement.get())); - - CassError code = cass_future_error_code(future.get()); - CassString message; - cass_future_error_message(future.get(), &message.data, &message.length); - BOOST_REQUIRE(code == CASS_ERROR_SERVER_INVALID_QUERY); - if (version.major == 1) { - BOOST_REQUIRE(std::string(message.data, message.length).find("Cannot execute query with bind variables") != std::string::npos); + // Handle deprecated and removed protocol versions [CASSANDRA-10146] + // https://issues.apache.org/jira/browse/CASSANDRA-10146 + CCM::CassVersion version = test_utils::get_version(); + if (version < "3.0.0") { + Version1Tests tester; + test_utils::execute_query(tester.session, "CREATE TABLE test (key int PRIMARY KEY, value int);"); + const char* insert_query = "INSERT INTO test (key, value) VALUES(?, ?);"; + test_utils::CassStatementPtr statement(cass_statement_new(insert_query, 2)); + + cass_statement_bind_int32(statement.get(), 0, 11); + cass_statement_bind_int32(statement.get(), 1, 99); + + test_utils::CassFuturePtr future(cass_session_execute(tester.session, statement.get())); + + CassError code = cass_future_error_code(future.get()); + CassString message; + cass_future_error_message(future.get(), &message.data, &message.length); + BOOST_REQUIRE(code == CASS_ERROR_SERVER_INVALID_QUERY); + if (version.major == 1) { + BOOST_REQUIRE(std::string(message.data, message.length).find("Cannot execute query with bind variables") != std::string::npos); + } else { + BOOST_REQUIRE(std::string(message.data, message.length).find("Invalid amount of bind variables") != std::string::npos); + } } else { - BOOST_REQUIRE(std::string(message.data, message.length).find("Invalid amount of bind variables") != std::string::npos); + std::cout << "Invalid Test for Cassandra v" << version.to_string() << ": Protocol v1 and v2 were dropped in v3.0.0" << std::endl; + BOOST_REQUIRE(true); } } diff --git a/test/integration_tests/src/test_version1_downgrade.cpp b/test/integration_tests/src/test_version1_downgrade.cpp index 62766a65a..2bebbf1c1 100644 --- a/test/integration_tests/src/test_version1_downgrade.cpp +++ b/test/integration_tests/src/test_version1_downgrade.cpp @@ -33,32 +33,38 @@ BOOST_AUTO_TEST_SUITE(version1_downgrade) BOOST_AUTO_TEST_CASE(query_after_downgrade) { - test_utils::CassLog::reset("does not support protocol version 2. Trying protocol version 1..."); + CCM::CassVersion version = test_utils::get_version(); + if (version < "2.0.0" ) { + test_utils::CassLog::reset("does not support protocol version 2. Trying protocol version 1..."); - size_t row_count; + size_t row_count; - { - test_utils::CassClusterPtr cluster(cass_cluster_new()); + { + test_utils::CassClusterPtr cluster(cass_cluster_new()); - boost::shared_ptr ccm(new CCM::Bridge("config.txt")); - if (ccm->create_cluster()) { - ccm->start_cluster(); - } + boost::shared_ptr ccm(new CCM::Bridge("config.txt")); + if (ccm->create_cluster()) { + ccm->start_cluster(); + } - test_utils::initialize_contact_points(cluster.get(), ccm->get_ip_prefix(), 1, 0); + test_utils::initialize_contact_points(cluster.get(), ccm->get_ip_prefix(), 1, 0); - cass_cluster_set_protocol_version(cluster.get(), 2); + cass_cluster_set_protocol_version(cluster.get(), 2); - test_utils::CassSessionPtr session(test_utils::create_session(cluster.get())); + test_utils::CassSessionPtr session(test_utils::create_session(cluster.get())); - test_utils::CassResultPtr result; - test_utils::execute_query(session.get(), "SELECT * FROM system.schema_keyspaces", &result); + test_utils::CassResultPtr result; + test_utils::execute_query(session.get(), "SELECT * FROM system.schema_keyspaces", &result); - row_count = cass_result_row_count(result.get()); - } + row_count = cass_result_row_count(result.get()); + } - BOOST_CHECK(row_count > 0); - BOOST_CHECK(test_utils::CassLog::message_count() > 0); + BOOST_CHECK(row_count > 0); + BOOST_CHECK(test_utils::CassLog::message_count() > 0); + } else { + std::cout << "Invalid Test for Cassandra v" << version.to_string() << ": Use Cassandra v1.2.x to test protocol v1 downgrade" << std::endl; + BOOST_REQUIRE(true); + } } BOOST_AUTO_TEST_SUITE_END() diff --git a/test/unit_tests/src/test_class_type_parser.cpp b/test/unit_tests/src/test_class_type_parser.cpp new file mode 100644 index 000000000..4a81c02c4 --- /dev/null +++ b/test/unit_tests/src/test_class_type_parser.cpp @@ -0,0 +1,316 @@ +/* + Copyright (c) 2014-2015 DataStax + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#ifdef STAND_ALONE +# define BOOST_TEST_MODULE cassandra +#endif + +#include "data_type_parser.hpp" + +#include + +BOOST_AUTO_TEST_SUITE(class_type_parser) + +BOOST_AUTO_TEST_CASE(simple) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + data_type = cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.InetAddressType", native_types); + BOOST_CHECK(data_type->value_type() == CASS_VALUE_TYPE_INET); + + data_type = cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.UTF8Type)", native_types); + BOOST_CHECK(data_type->value_type() == CASS_VALUE_TYPE_TEXT); + + data_type = cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.UTF8Type)", native_types); + BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_LIST); + + cass::CollectionType::ConstPtr collection + = static_cast(data_type); + BOOST_REQUIRE(collection->types().size() == 1); + BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); +} + +BOOST_AUTO_TEST_CASE(invalid) +{ + cass_log_set_level(CASS_LOG_DISABLED); + + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + // Premature end of string + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType", native_types)); + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType(", native_types)); + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType(blah", native_types)); + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType(blah,", native_types)); + + // Empty + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType()", native_types)); + + // Invalid hex + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType(blah,ZZZZ", native_types)); + + // Missing ':' + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType(" + "foo,61646472657373," + "737472656574org.apache.cassandra.db.marshal.UTF8Type)", native_types)); + + // Premature end of string + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType", native_types)); + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(", native_types)); + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type", native_types)); + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type,", native_types)); + + // Empty + BOOST_CHECK(!cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType()", native_types)); +} + +BOOST_AUTO_TEST_CASE(udt) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::DataType::ConstPtr data_type + = cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.UserType(" + "foo,61646472657373," + "737472656574:org.apache.cassandra.db.marshal.UTF8Type," + "7a6970636f6465:org.apache.cassandra.db.marshal.Int32Type," + "70686f6e6573:org.apache.cassandra.db.marshal.SetType(" + "org.apache.cassandra.db.marshal.UserType(foo,70686f6e65,6e616d65:org.apache.cassandra.db.marshal.UTF8Type,6e756d626572:org.apache.cassandra.db.marshal.UTF8Type)))", + native_types); + + BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_UDT); + + // Check external UDT + + cass::UserType::ConstPtr udt(data_type); + + BOOST_CHECK(udt->keyspace() == "foo"); + BOOST_CHECK(udt->type_name() == "address"); + BOOST_REQUIRE(udt->fields().size() == 3); + + cass::UserType::FieldVec::const_iterator i; + + i = udt->fields().begin(); + + BOOST_CHECK(i->name == "street"); + BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_TEXT); + + ++i; + + BOOST_CHECK(i->name == "zipcode"); + BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_INT); + + ++i; + + BOOST_CHECK(i->name == "phones"); + BOOST_REQUIRE(i->type->value_type() == CASS_VALUE_TYPE_SET); + + cass::CollectionType::ConstPtr collection + = static_cast(i->type); + + BOOST_REQUIRE(collection->types().size() == 1); + BOOST_REQUIRE(collection->types()[0]->value_type() == CASS_VALUE_TYPE_UDT); + + // Check internal UDT + + udt = static_cast(collection->types()[0]); + + BOOST_CHECK(udt->keyspace() == "foo"); + BOOST_CHECK(udt->type_name() == "phone"); + BOOST_REQUIRE(udt->fields().size() == 2); + + i = udt->fields().begin(); + + BOOST_CHECK(i->name == "name"); + BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_TEXT); + + ++i; + + BOOST_CHECK(i->name == "number"); + BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_TEXT); +} + +BOOST_AUTO_TEST_CASE(tuple) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::DataType::ConstPtr data_type + = cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.TupleType(" + "org.apache.cassandra.db.marshal.Int32Type," + "org.apache.cassandra.db.marshal.UTF8Type," + "org.apache.cassandra.db.marshal.FloatType)", native_types); + + BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_TUPLE); + + cass::TupleType::ConstPtr tuple = static_cast(data_type); + + BOOST_REQUIRE(tuple->types().size() == 3); + + BOOST_REQUIRE(tuple->types()[0]->value_type() == CASS_VALUE_TYPE_INT); + BOOST_REQUIRE(tuple->types()[1]->value_type() == CASS_VALUE_TYPE_TEXT); + BOOST_REQUIRE(tuple->types()[2]->value_type() == CASS_VALUE_TYPE_FLOAT); +} + +BOOST_AUTO_TEST_CASE(nested_collections) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::DataType::ConstPtr data_type + = cass::DataTypeClassNameParser::parse_one("org.apache.cassandra.db.marshal.MapType(" + "org.apache.cassandra.db.marshal.UTF8Type," + "org.apache.cassandra.db.marshal.FrozenType(" + "org.apache.cassandra.db.marshal.MapType(" + "org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.Int32Type)))", native_types); + + BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_MAP); + + cass::CollectionType::ConstPtr collection + = static_cast(data_type); + + BOOST_REQUIRE(collection->types().size() == 2); + + BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); + + BOOST_REQUIRE(collection->types()[1]->value_type() == CASS_VALUE_TYPE_MAP); + + cass::CollectionType::ConstPtr nested_collection + = static_cast(collection->types()[1]); + + BOOST_REQUIRE(nested_collection->types().size() == 2); + BOOST_CHECK(nested_collection->types()[0]->value_type() == CASS_VALUE_TYPE_INT); + BOOST_CHECK(nested_collection->types()[1]->value_type() == CASS_VALUE_TYPE_INT); +} + +BOOST_AUTO_TEST_CASE(composite) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::SharedRefPtr result + = cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(" + "org.apache.cassandra.db.marshal.AsciiType," + "org.apache.cassandra.db.marshal.Int32Type)", native_types); + + BOOST_CHECK(result->is_composite()); + + BOOST_REQUIRE(result->types().size() == 2); + BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_ASCII); + BOOST_CHECK(result->types()[1]->value_type() == CASS_VALUE_TYPE_INT); + + BOOST_REQUIRE(result->reversed().size() == 2); + BOOST_CHECK(result->reversed()[0] == false); + BOOST_CHECK(result->reversed()[1] == false); + + BOOST_CHECK(result->collections().empty()); +} + +BOOST_AUTO_TEST_CASE(not_composite) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::SharedRefPtr result + = cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.InetAddressType", native_types); + + BOOST_REQUIRE(result->types().size() == 1); + BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_INET); + + BOOST_REQUIRE(result->reversed().size() == 1); + BOOST_CHECK(result->reversed()[0] == false); +} + +BOOST_AUTO_TEST_CASE(composite_with_reversed) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::SharedRefPtr result + = cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(" + "org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.AsciiType)," + "org.apache.cassandra.db.marshal.Int32Type)", native_types); + + BOOST_CHECK(result->is_composite()); + + BOOST_REQUIRE(result->types().size() == 2); + BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_ASCII); + BOOST_CHECK(result->types()[1]->value_type() == CASS_VALUE_TYPE_INT); + + BOOST_REQUIRE(result->reversed().size() == 2); + BOOST_CHECK(result->reversed()[0] == true); + BOOST_CHECK(result->reversed()[1] == false); + + BOOST_CHECK(result->collections().empty()); +} + +BOOST_AUTO_TEST_CASE(composite_with_collections) +{ + cass::NativeDataTypes native_types; + native_types.init_class_names(); + + cass::SharedRefPtr result + = cass::DataTypeClassNameParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(" + "org.apache.cassandra.db.marshal.Int32Type, " + "org.apache.cassandra.db.marshal.UTF8Type," + "org.apache.cassandra.db.marshal.ColumnToCollectionType(" + "6162:org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.Int32Type)," + "4A4b4C4D4e4F:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type)," + "6A6b6C6D6e6F:org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type, org.apache.cassandra.db.marshal.LongType)" + "))", native_types); + + BOOST_CHECK(result->is_composite()); + + BOOST_REQUIRE(result->types().size() == 2); + BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_INT); + BOOST_CHECK(result->types()[1]->value_type() == CASS_VALUE_TYPE_TEXT); + + BOOST_REQUIRE(result->reversed().size() == 2); + BOOST_CHECK(result->reversed()[0] == false); + BOOST_CHECK(result->reversed()[1] == false); + + BOOST_REQUIRE(result->collections().size() == 3); + + cass::ParseResult::CollectionMap::const_iterator i; + + i = result->collections().find("ab"); + cass::CollectionType::ConstPtr collection; + BOOST_REQUIRE(i != result->collections().end()); + BOOST_REQUIRE(i->second->value_type() == CASS_VALUE_TYPE_LIST); + collection = static_cast(i->second); + BOOST_REQUIRE(collection->types().size() == 1); + BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_INT); + + i = result->collections().find("JKLMNO"); + BOOST_REQUIRE(i != result->collections().end()); + BOOST_CHECK(i->second->value_type() == CASS_VALUE_TYPE_SET); + collection = static_cast(i->second); + BOOST_REQUIRE(collection->types().size() == 1); + BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); + + i = result->collections().find("jklmno"); + BOOST_REQUIRE(i != result->collections().end()); + BOOST_CHECK(i->second->value_type() == CASS_VALUE_TYPE_MAP); + collection = static_cast(i->second); + BOOST_REQUIRE(collection->types().size() == 2); + BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); + BOOST_CHECK(collection->types()[1]->value_type() == CASS_VALUE_TYPE_BIGINT); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/unit_tests/src/test_cql_type_parser.cpp b/test/unit_tests/src/test_cql_type_parser.cpp new file mode 100644 index 000000000..977d7dbfd --- /dev/null +++ b/test/unit_tests/src/test_cql_type_parser.cpp @@ -0,0 +1,213 @@ +/* + Copyright (c) 2014-2015 DataStax + + Licensed under the Apache License, Version 2.0 (the "License"); + you may not use this file except in compliance with the License. + You may obtain a copy of the License at + + http://www.apache.org/licenses/LICENSE-2.0 + + Unless required by applicable law or agreed to in writing, software + distributed under the License is distributed on an "AS IS" BASIS, + WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + See the License for the specific language governing permissions and + limitations under the License. +*/ + +#ifdef STAND_ALONE +# define BOOST_TEST_MODULE cassandra +#endif + +#include "data_type_parser.hpp" +#include "metadata.hpp" + +#include + +BOOST_AUTO_TEST_SUITE(cql_type_parser) + +BOOST_AUTO_TEST_CASE(simple) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_cql_names(); + + cass::KeyspaceMetadata keyspace("keyspace1"); + + data_type = cass::DataTypeCqlNameParser::parse("ascii", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_ASCII); + + data_type = cass::DataTypeCqlNameParser::parse("bigint", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_BIGINT); + + data_type = cass::DataTypeCqlNameParser::parse("blob", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_BLOB); + + data_type = cass::DataTypeCqlNameParser::parse("boolean", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_BOOLEAN); + + data_type = cass::DataTypeCqlNameParser::parse("counter", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_COUNTER); + + data_type = cass::DataTypeCqlNameParser::parse("date", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_DATE); + + data_type = cass::DataTypeCqlNameParser::parse("decimal", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_DECIMAL); + + data_type = cass::DataTypeCqlNameParser::parse("double", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_DOUBLE); + + data_type = cass::DataTypeCqlNameParser::parse("float", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_FLOAT); + + data_type = cass::DataTypeCqlNameParser::parse("inet", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_INET); + + data_type = cass::DataTypeCqlNameParser::parse("int", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_INT); + + data_type = cass::DataTypeCqlNameParser::parse("smallint", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_SMALL_INT); + + data_type = cass::DataTypeCqlNameParser::parse("time", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_TIME); + + data_type = cass::DataTypeCqlNameParser::parse("timestamp", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_TIMESTAMP); + + data_type = cass::DataTypeCqlNameParser::parse("timeuuid", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_TIMEUUID); + + data_type = cass::DataTypeCqlNameParser::parse("tinyint", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_TINY_INT); + + data_type = cass::DataTypeCqlNameParser::parse("text", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_TEXT); + + data_type = cass::DataTypeCqlNameParser::parse("uuid", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_UUID); + + data_type = cass::DataTypeCqlNameParser::parse("varchar", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_VARCHAR); + + data_type = cass::DataTypeCqlNameParser::parse("varint", native_types, &keyspace); + BOOST_CHECK_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_VARINT); +} + +BOOST_AUTO_TEST_CASE(collections) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_cql_names(); + + cass::KeyspaceMetadata keyspace("keyspace1"); + + data_type = cass::DataTypeCqlNameParser::parse("list", native_types, &keyspace); + BOOST_REQUIRE_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_LIST); + cass::CollectionType::ConstPtr list = static_cast(data_type); + BOOST_REQUIRE_EQUAL(list->types().size(), 1); + BOOST_CHECK_EQUAL(list->types()[0]->value_type(), CASS_VALUE_TYPE_INT); + + data_type = cass::DataTypeCqlNameParser::parse("set", native_types, &keyspace); + BOOST_REQUIRE_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_SET); + cass::CollectionType::ConstPtr set = static_cast(data_type); + BOOST_REQUIRE_EQUAL(set->types().size(), 1); + BOOST_CHECK_EQUAL(set->types()[0]->value_type(), CASS_VALUE_TYPE_INT); + + data_type = cass::DataTypeCqlNameParser::parse("map", native_types, &keyspace); + BOOST_REQUIRE_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_MAP); + cass::CollectionType::ConstPtr map = static_cast(data_type); + BOOST_REQUIRE_EQUAL(map->types().size(), 2); + BOOST_CHECK_EQUAL(map->types()[0]->value_type(), CASS_VALUE_TYPE_INT); + BOOST_CHECK_EQUAL(map->types()[1]->value_type(), CASS_VALUE_TYPE_TEXT); +} + +BOOST_AUTO_TEST_CASE(tuple) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_cql_names(); + + cass::KeyspaceMetadata keyspace("keyspace1"); + + data_type = cass::DataTypeCqlNameParser::parse("tuple", native_types, &keyspace); + BOOST_REQUIRE_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_TUPLE); + cass::CollectionType::ConstPtr tuple = static_cast(data_type); + BOOST_REQUIRE_EQUAL(tuple->types().size(), 3); + BOOST_CHECK_EQUAL(tuple->types()[0]->value_type(), CASS_VALUE_TYPE_INT); + BOOST_CHECK_EQUAL(tuple->types()[1]->value_type(), CASS_VALUE_TYPE_BIGINT); + BOOST_CHECK_EQUAL(tuple->types()[2]->value_type(), CASS_VALUE_TYPE_TEXT); +} + +BOOST_AUTO_TEST_CASE(udt) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_cql_names(); + + cass::KeyspaceMetadata keyspace("keyspace1"); + + BOOST_CHECK(keyspace.user_types().empty()); + + data_type = cass::DataTypeCqlNameParser::parse("type1", native_types, &keyspace); + + BOOST_REQUIRE_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_UDT); + cass::UserType::ConstPtr udt = static_cast(data_type); + BOOST_CHECK_EQUAL(udt->type_name(), "type1"); + BOOST_CHECK_EQUAL(udt->keyspace(), "keyspace1"); + + BOOST_CHECK(!keyspace.user_types().empty()); +} + +BOOST_AUTO_TEST_CASE(frozen) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_cql_names(); + + cass::KeyspaceMetadata keyspace("keyspace1"); + + data_type = cass::DataTypeCqlNameParser::parse("frozen>", native_types, &keyspace); + BOOST_REQUIRE_EQUAL(data_type->value_type(), CASS_VALUE_TYPE_LIST); + cass::CollectionType::ConstPtr list = static_cast(data_type); + BOOST_REQUIRE_EQUAL(list->types().size(), 1); + BOOST_CHECK_EQUAL(list->types()[0]->value_type(), CASS_VALUE_TYPE_INT); +} + +BOOST_AUTO_TEST_CASE(invalid) +{ + cass::DataType::ConstPtr data_type; + + cass::NativeDataTypes native_types; + native_types.init_cql_names(); + + cass::KeyspaceMetadata keyspace("keyspace1"); + + // Invalid number of parameters + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("list<>", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("set<>", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("map<>", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("tuple<>", native_types, &keyspace)); + + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("list", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("set", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("map", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("map", native_types, &keyspace)); + + // Invalid brackets + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("list<", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("list>", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("<>", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("<", native_types, &keyspace)); + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse(">", native_types, &keyspace)); + + // Empty + BOOST_CHECK(!cass::DataTypeCqlNameParser::parse("", native_types, &keyspace)); +} + +BOOST_AUTO_TEST_SUITE_END() diff --git a/test/unit_tests/src/test_type_parser.cpp b/test/unit_tests/src/test_type_parser.cpp deleted file mode 100644 index 6942bd1bb..000000000 --- a/test/unit_tests/src/test_type_parser.cpp +++ /dev/null @@ -1,288 +0,0 @@ -/* - Copyright (c) 2014-2015 DataStax - - Licensed under the Apache License, Version 2.0 (the "License"); - you may not use this file except in compliance with the License. - You may obtain a copy of the License at - - http://www.apache.org/licenses/LICENSE-2.0 - - Unless required by applicable law or agreed to in writing, software - distributed under the License is distributed on an "AS IS" BASIS, - WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - See the License for the specific language governing permissions and - limitations under the License. -*/ - -#ifdef STAND_ALONE -# define BOOST_TEST_MODULE cassandra -#endif - -#include "type_parser.hpp" - -#include - -BOOST_AUTO_TEST_SUITE(type_parser) - -BOOST_AUTO_TEST_CASE(simple) -{ - cass::SharedRefPtr data_type; - - data_type = cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.InetAddressType"); - BOOST_CHECK(data_type->value_type() == CASS_VALUE_TYPE_INET); - - data_type = cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.UTF8Type)"); - BOOST_CHECK(data_type->value_type() == CASS_VALUE_TYPE_TEXT); - - data_type = cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.UTF8Type)"); - BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_LIST); - - cass::SharedRefPtr collection - = static_cast >(data_type); - BOOST_REQUIRE(collection->types().size() == 1); - BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); -} - -BOOST_AUTO_TEST_CASE(invalid) -{ - cass_log_set_level(CASS_LOG_DISABLED); - - // Premature end of string - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType")); - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType(")); - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType(blah")); - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType(blah,")); - - // Empty - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType()")); - - // Invalid hex - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType(blah,ZZZZ")); - - // Missing ':' - BOOST_CHECK(!cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType(" - "foo,61646472657373," - "737472656574org.apache.cassandra.db.marshal.UTF8Type)")); - - // Premature end of string - BOOST_CHECK(!cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType")); - BOOST_CHECK(!cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(")); - BOOST_CHECK(!cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type")); - BOOST_CHECK(!cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(org.apache.cassandra.db.marshal.UTF8Type,")); - - // Empty - BOOST_CHECK(!cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType()")); -} - -BOOST_AUTO_TEST_CASE(udt) -{ - cass::SharedRefPtr data_type - = cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.UserType(" - "foo,61646472657373," - "737472656574:org.apache.cassandra.db.marshal.UTF8Type," - "7a6970636f6465:org.apache.cassandra.db.marshal.Int32Type," - "70686f6e6573:org.apache.cassandra.db.marshal.SetType(" - "org.apache.cassandra.db.marshal.UserType(foo,70686f6e65,6e616d65:org.apache.cassandra.db.marshal.UTF8Type,6e756d626572:org.apache.cassandra.db.marshal.UTF8Type)))"); - - BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_UDT); - - // Check external UDT - - cass::SharedRefPtr udt(data_type); - - BOOST_CHECK(udt->keyspace() == "foo"); - BOOST_CHECK(udt->type_name() == "address"); - BOOST_REQUIRE(udt->fields().size() == 3); - - cass::UserType::FieldVec::const_iterator i; - - i = udt->fields().begin(); - - BOOST_CHECK(i->name == "street"); - BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_TEXT); - - ++i; - - BOOST_CHECK(i->name == "zipcode"); - BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_INT); - - ++i; - - BOOST_CHECK(i->name == "phones"); - BOOST_REQUIRE(i->type->value_type() == CASS_VALUE_TYPE_SET); - - cass::SharedRefPtr collection - = static_cast >(i->type); - - BOOST_REQUIRE(collection->types().size() == 1); - BOOST_REQUIRE(collection->types()[0]->value_type() == CASS_VALUE_TYPE_UDT); - - // Check internal UDT - - udt = static_cast >(collection->types()[0]); - - BOOST_CHECK(udt->keyspace() == "foo"); - BOOST_CHECK(udt->type_name() == "phone"); - BOOST_REQUIRE(udt->fields().size() == 2); - - i = udt->fields().begin(); - - BOOST_CHECK(i->name == "name"); - BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_TEXT); - - ++i; - - BOOST_CHECK(i->name == "number"); - BOOST_CHECK(i->type->value_type() == CASS_VALUE_TYPE_TEXT); -} - -BOOST_AUTO_TEST_CASE(tuple) -{ - cass::SharedRefPtr data_type - = cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.TupleType(" - "org.apache.cassandra.db.marshal.Int32Type," - "org.apache.cassandra.db.marshal.UTF8Type," - "org.apache.cassandra.db.marshal.FloatType)"); - - BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_TUPLE); - - cass::SharedRefPtr tuple = static_cast >(data_type); - - BOOST_REQUIRE(tuple->types().size() == 3); - - BOOST_REQUIRE(tuple->types()[0]->value_type() == CASS_VALUE_TYPE_INT); - BOOST_REQUIRE(tuple->types()[1]->value_type() == CASS_VALUE_TYPE_TEXT); - BOOST_REQUIRE(tuple->types()[2]->value_type() == CASS_VALUE_TYPE_FLOAT); -} - -BOOST_AUTO_TEST_CASE(nested_collections) -{ - cass::SharedRefPtr data_type - = cass::TypeParser::parse_one("org.apache.cassandra.db.marshal.MapType(" - "org.apache.cassandra.db.marshal.UTF8Type," - "org.apache.cassandra.db.marshal.FrozenType(" - "org.apache.cassandra.db.marshal.MapType(" - "org.apache.cassandra.db.marshal.Int32Type,org.apache.cassandra.db.marshal.Int32Type)))"); - - BOOST_REQUIRE(data_type->value_type() == CASS_VALUE_TYPE_MAP); - - cass::SharedRefPtr collection - = static_cast >(data_type); - - BOOST_REQUIRE(collection->types().size() == 2); - - BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); - - BOOST_REQUIRE(collection->types()[1]->value_type() == CASS_VALUE_TYPE_MAP); - - cass::SharedRefPtr nested_collection - = static_cast >(collection->types()[1]); - - BOOST_REQUIRE(nested_collection->types().size() == 2); - BOOST_CHECK(nested_collection->types()[0]->value_type() == CASS_VALUE_TYPE_INT); - BOOST_CHECK(nested_collection->types()[1]->value_type() == CASS_VALUE_TYPE_INT); -} - -BOOST_AUTO_TEST_CASE(composite) -{ - cass::SharedRefPtr result - = cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(" - "org.apache.cassandra.db.marshal.AsciiType," - "org.apache.cassandra.db.marshal.Int32Type)"); - - BOOST_CHECK(result->is_composite()); - - BOOST_REQUIRE(result->types().size() == 2); - BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_ASCII); - BOOST_CHECK(result->types()[1]->value_type() == CASS_VALUE_TYPE_INT); - - BOOST_REQUIRE(result->reversed().size() == 2); - BOOST_CHECK(result->reversed()[0] == false); - BOOST_CHECK(result->reversed()[1] == false); - - BOOST_CHECK(result->collections().empty()); -} - -BOOST_AUTO_TEST_CASE(not_composite) -{ - cass::SharedRefPtr result - = cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.InetAddressType"); - - BOOST_REQUIRE(result->types().size() == 1); - BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_INET); - - BOOST_REQUIRE(result->reversed().size() == 1); - BOOST_CHECK(result->reversed()[0] == false); -} - -BOOST_AUTO_TEST_CASE(composite_with_reversed) -{ - cass::SharedRefPtr result - = cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(" - "org.apache.cassandra.db.marshal.ReversedType(org.apache.cassandra.db.marshal.AsciiType)," - "org.apache.cassandra.db.marshal.Int32Type)"); - - BOOST_CHECK(result->is_composite()); - - BOOST_REQUIRE(result->types().size() == 2); - BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_ASCII); - BOOST_CHECK(result->types()[1]->value_type() == CASS_VALUE_TYPE_INT); - - BOOST_REQUIRE(result->reversed().size() == 2); - BOOST_CHECK(result->reversed()[0] == true); - BOOST_CHECK(result->reversed()[1] == false); - - BOOST_CHECK(result->collections().empty()); -} - -BOOST_AUTO_TEST_CASE(composite_with_collections) -{ - cass::SharedRefPtr result - = cass::TypeParser::parse_with_composite("org.apache.cassandra.db.marshal.CompositeType(" - "org.apache.cassandra.db.marshal.Int32Type, " - "org.apache.cassandra.db.marshal.UTF8Type," - "org.apache.cassandra.db.marshal.ColumnToCollectionType(" - "6162:org.apache.cassandra.db.marshal.ListType(org.apache.cassandra.db.marshal.Int32Type)," - "4A4b4C4D4e4F:org.apache.cassandra.db.marshal.SetType(org.apache.cassandra.db.marshal.UTF8Type)," - "6A6b6C6D6e6F:org.apache.cassandra.db.marshal.MapType(org.apache.cassandra.db.marshal.UTF8Type, org.apache.cassandra.db.marshal.LongType)" - "))"); - - BOOST_CHECK(result->is_composite()); - - BOOST_REQUIRE(result->types().size() == 2); - BOOST_CHECK(result->types()[0]->value_type() == CASS_VALUE_TYPE_INT); - BOOST_CHECK(result->types()[1]->value_type() == CASS_VALUE_TYPE_TEXT); - - BOOST_REQUIRE(result->reversed().size() == 2); - BOOST_CHECK(result->reversed()[0] == false); - BOOST_CHECK(result->reversed()[1] == false); - - BOOST_REQUIRE(result->collections().size() == 3); - - cass::ParseResult::CollectionMap::const_iterator i; - - i = result->collections().find("ab"); - cass::SharedRefPtr collection; - BOOST_REQUIRE(i != result->collections().end()); - BOOST_REQUIRE(i->second->value_type() == CASS_VALUE_TYPE_LIST); - collection = static_cast >(i->second); - BOOST_REQUIRE(collection->types().size() == 1); - BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_INT); - - i = result->collections().find("JKLMNO"); - BOOST_REQUIRE(i != result->collections().end()); - BOOST_CHECK(i->second->value_type() == CASS_VALUE_TYPE_SET); - collection = static_cast >(i->second); - BOOST_REQUIRE(collection->types().size() == 1); - BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); - - i = result->collections().find("jklmno"); - BOOST_REQUIRE(i != result->collections().end()); - BOOST_CHECK(i->second->value_type() == CASS_VALUE_TYPE_MAP); - collection = static_cast >(i->second); - BOOST_REQUIRE(collection->types().size() == 2); - BOOST_CHECK(collection->types()[0]->value_type() == CASS_VALUE_TYPE_TEXT); - BOOST_CHECK(collection->types()[1]->value_type() == CASS_VALUE_TYPE_BIGINT); -} - -BOOST_AUTO_TEST_SUITE_END()