Cassandra Documentation

Version:

You are viewing the documentation for a prerelease version.

Functions

CQL supports 2 main categories of functions:

In both cases, CQL provides a number of native "hard-coded" functions as well as the ability to create new user-defined functions.

By default, the use of user-defined functions is disabled by default for security concerns (even when enabled, the execution of user-defined functions is sandboxed and a "rogue" function should not be allowed to do evil, but no sandbox is perfect so using user-defined functions is opt-in). See the user_defined_functions_enabled in cassandra.yaml to enable them.

A function is identifier by its name:

function_name ::= [ keyspace_name'.' ] name

Scalar functions

Native functions

Cast

The cast function can be used to converts one native datatype to another.

The following table describes the conversions supported by the cast function. Cassandra will silently ignore any cast converting a datatype into its own datatype.

From To

ascii

text, varchar

bigint

tinyint, smallint, int, float, double, decimal, varint, text, varchar

boolean

text, varchar

counter

tinyint, smallint, int, bigint, float, double, decimal, varint, text, varchar

date

timestamp

decimal

tinyint, smallint, int, bigint, float, double, varint, text, varchar

double

tinyint, smallint, int, bigint, float, decimal, varint, text, varchar

float

tinyint, smallint, int, bigint, double, decimal, varint, text, varchar

inet

text, varchar

int

tinyint, smallint, bigint, float, double, decimal, varint, text, varchar

smallint

tinyint, int, bigint, float, double, decimal, varint, text, varchar

time

text, varchar

timestamp

date, text, varchar

timeuuid

timestamp, date, text, varchar

tinyint

tinyint, smallint, int, bigint, float, double, decimal, varint, text, varchar

uuid

text, varchar

varint

tinyint, smallint, int, bigint, float, double, decimal, text, varchar

The conversions rely strictly on Java’s semantics. For example, the double value 1 will be converted to the text value '1.0'. For instance:

SELECT avg(cast(count as double)) FROM myTable

Token

The token function computes the token for a given partition key. The exact signature of the token function depends on the table concerned and the partitioner used by the cluster.

The type of the arguments of the token depend on the partition key column type. The returned type depends on the defined partitioner:

Partitioner Returned type

Murmur3Partitioner

bigint

RandomPartitioner

varint

ByteOrderedPartitioner

blob

For example, consider the following table:

CREATE TABLE users (
    userid text PRIMARY KEY,
    username text,
);

The table uses the default Murmur3Partitioner. The token function uses the single argument text, because the partition key is userid of text type. The returned type will be bigint.

Uuid

The uuid function takes no parameters and generates a random type 4 uuid suitable for use in INSERT or UPDATE statements.

Timeuuid functions

now

The now function takes no arguments and generates, on the coordinator node, a new unique timeuuid at the time the function is invoked. Note that this method is useful for insertion but is largely non-sensical in WHERE clauses.

For example, a query of the form:

SELECT * FROM myTable WHERE t = now();

will not return a result, by design, since the value returned by now() is guaranteed to be unique.

current_timeuuid is an alias of now.

min_timeuuid and max_timeuuid

The min_timeuuid function takes a timestamp value t, either a timestamp or a date string. It returns a fake timeuuid corresponding to the smallest possible timeuuid for timestamp t. The max_timeuuid works similarly, but returns the largest possible timeuuid.

For example:

SELECT * FROM myTable
 WHERE t > max_timeuuid('2013-01-01 00:05+0000')
   AND t < min_timeuuid('2013-02-02 10:00+0000');

will select all rows where the timeuuid column t is later than '2013-01-01 00:05+0000' and earlier than '2013-02-02 10:00+0000'. The clause t >= maxTimeuuid('2013-01-01 00:05+0000') would still not select a timeuuid generated exactly at '2013-01-01 00:05+0000', and is essentially equivalent to t > maxTimeuuid('2013-01-01 00:05+0000').

The values generated by min_timeuuid and max_timeuuid are called fake UUID because they do no respect the time-based UUID generation process specified by the IETF RFC 4122. In particular, the value returned by these two methods will not be unique. Thus, only use these methods for querying, not for insertion, to prevent possible data overwriting.

Datetime functions

Retrieving the current date/time

The following functions can be used to retrieve the date/time at the time where the function is invoked:

Function name Output type

current_timestamp

timestamp

current_date

date

current_time

time

current_timeuuid

timeUUID

For example the last two days of data can be retrieved using:

SELECT * FROM myTable WHERE date >= current_date() - 2d;
Time conversion functions

A number of functions are provided to convert a timeuuid, a timestamp or a date into another native type.

Function name Input type Description

to_date

timeuuid

Converts the timeuuid argument into a date type

to_date

timestamp

Converts the timestamp argument into a date type

to_timestamp

timeuuid

Converts the timeuuid argument into a timestamp type

to_timestamp

date

Converts the date argument into a timestamp type

to_unix_timestamp

timeuuid

Converts the timeuuid argument into a bigInt raw value

to_unix_timestamp

timestamp

Converts the timestamp argument into a bigInt raw value

to_unix_timestamp

date

Converts the date argument into a bigInt raw value

Blob conversion functions

A number of functions are provided to convert the native types into binary data, or a blob. For every type supported by CQL, the function type_as_blob takes a argument of type type and returns it as a blob. Conversely, the function blob_as_type takes a 64-bit blob argument and converts it to a bigint value. For example, bigint_as_blob(3) returns 0x0000000000000003 and blob_as_bigint(0x0000000000000003) returns 3.

Math Functions

Cql provides the following math functions: abs, exp, log, log10, and round. The return type for these functions is always the same as the input type.

Function name Description

abs

Returns the absolute value of the input.

exp

Returns the number e to the power of the input.

log

Returns the natural log of the input.

log10

Returns the log base 10 of the input.

round

Rounds the input to the nearest whole number using rounding mode HALF_UP.

Collection functions

A number of functions are provided to operate on collection columns.

Function name Input type Description

map_keys

map

Gets the keys of the map argument, returned as a set.

map_values

map

Gets the values of the map argument, returned as a list.

collection_count

map, set or list

Gets the number of elements in the collection argument.

collection_min

set or list

Gets the minimum element in the collection argument.

collection_max

set or list

Gets the maximum element in the collection argument.

collection_sum

numeric set or list

Computes the sum of the elements in the collection argument. The returned value is of the same type as the input collection elements, so there is a risk of overflowing the data type if the sum of the values exceeds the maximum value that the type can represent.

collection_avg

numeric set or list

Computes the average of the elements in the collection argument. The average of an empty collection returns zero. The returned value is of the same type as the input collection elements, which might include rounding and truncations. For example collection_avg([1, 2]) returns 1 instead of 1.5.

Data masking functions

A number of functions allow to obscure the real contents of a column containing sensitive data.

Function Description

mask_null(value)

Replaces the first argument with a null column. The returned value is always a non-existent column, and not a not-null column representing a null value.

Examples:

mask_null('Alice')null

mask_null(123)null

mask_default(value)

Replaces its argument by an arbitrary, fixed default value of the same type. This will be **** for text values, zero for numeric values, false for booleans, etc.

Variable-length multi-valued types such as lists, sets and maps are masked as empty collections.

Fixed-length multi-valued types such as tuples, user-defined types (UDTs) and vectors are masked by replacing each of their values by the default masking value of the value type.

Examples:

mask_default('Alice')'****'

mask_default(123)0

mask_default((list<int>) [1, 2, 3])[]

mask_default((vector<int, 3>) [1, 2, 3])[0, 0, 0]

mask_replace(value, replacement])

Replaces the first argument by the replacement value on the second argument. The replacement value needs to have the same type as the replaced value.

Examples:

mask_replace('Alice', 'REDACTED')'REDACTED'

mask_replace(123, -1)-1

mask_inner(value, begin, end, [padding])

Returns a copy of the first text, varchar or ascii argument, replacing each character except the first and last ones by a padding character. The second and third arguments are the size of the exposed prefix and suffix. The optional fourth argument is the padding character, \* by default.

Examples:

mask_inner('Alice', 1, 2)'Ace'

mask_inner('Alice', 1, null)'A'

mask_inner('Alice', null, 2)'*ce'

mask_inner('Alice', 2, 1, '#')'Al##e'

mask_outer(value, begin, end, [padding])

Returns a copy of the first text, varchar or ascii argument, replacing the first and last character by a padding character. The second and third arguments are the size of the exposed prefix and suffix. The optional fourth argument is the padding character, \* by default.

Examples:

mask_outer('Alice', 1, 2)'*li'

mask_outer('Alice', 1, null)'*lice'

mask_outer('Alice', null, 2)'Ali'

mask_outer('Alice', 2, 1, '#')'##ic#'

mask_hash(value, [algorithm])

Returns a blob containing the hash of the first argument. The optional second argument is the hashing algorithm to be used, according the available Java security provider. The default hashing algorithm is SHA-256.

Examples:

mask_hash('Alice')

mask_hash('Alice', 'SHA-512')

Vector similarity functions

A number of functions allow to obtain the similarity score between vectors of floats.

Function Description

similarity_cosine(vector, vector)

Calculates the cosine similarity score between two float vectors of the same dimension.

Examples:

similarity_cosine([0.1, 0.2], null)null

similarity_cosine([0.1, 0.2], [0.1, 0.2])1

similarity_cosine([0.1, 0.2], [-0.1, -0.2])0

similarity_cosine([0.1, 0.2], [0.9, 0.8])0.964238

similarity_euclidean(vector, vector)

Calculates the euclidian distance between two float vectors of the same dimension.

Examples:

similarity_euclidean([0.1, 0.2], null)null

similarity_euclidean([0.1, 0.2], [0.1, 0.2])1

similarity_euclidean([0.1, 0.2], [-0.1, -0.2])0.833333

similarity_euclidean([0.1, 0.2], [0.9, 0.8])0.5

similarity_dot_product(vector, vector)

Calculates the dot product between two float vectors of the same dimension.

Examples:

similarity_dot_product([0.1, 0.2], null)null

similarity_dot_product([0.1, 0.2], [0.1, 0.2])0.525

similarity_dot_product([0.1, 0.2], [-0.1, -0.2])0.475

similarity_dot_product([0.1, 0.2], [0.9, 0.8])0.625

User-defined functions

User-defined functions (UDFs) execute user-provided code in Cassandra. By default, Cassandra supports defining functions in Java.

UDFs are part of the Cassandra schema, and are automatically propagated to all nodes in the cluster. UDFs can be overloaded, so that multiple UDFs with different argument types can have the same function name.

JavaScript user-defined functions have been deprecated in Cassandra 4.1. In preparation for Cassandra 5.0, their removal is already in progress. For more information - CASSANDRA-17281, CASSANDRA-18252.

For example:

CREATE FUNCTION sample ( arg int ) ...;
CREATE FUNCTION sample ( arg text ) ...;

UDFs are susceptible to all of the normal problems with the chosen programming language. Accordingly, implementations should be safe against null pointer exceptions, illegal arguments, or any other potential source of exceptions. An exception during function execution will result in the entire statement failing. Valid queries for UDF use are SELECT, INSERT and UPDATE statements.

Complex types like collections, tuple types and user-defined types are valid argument and return types in UDFs. Tuple types and user-defined types use the DataStax Java Driver conversion functions. Please see the Java Driver documentation for details on handling tuple types and user-defined types.

Arguments for functions can be literals or terms. Prepared statement placeholders can be used, too.

Note the use the double dollar-sign syntax to enclose the UDF source code.

For example:

CREATE FUNCTION some_function ( arg int )
    RETURNS NULL ON NULL INPUT
    RETURNS int
    LANGUAGE java
    AS $$ return arg; $$;

SELECT some_function(column) FROM atable ...;
UPDATE atable SET col = some_function(?) ...;

CREATE TYPE custom_type (txt text, i int);
CREATE FUNCTION fct_using_udt ( udtarg frozen )
    RETURNS NULL ON NULL INPUT
    RETURNS text
    LANGUAGE java
    AS $$ return udtarg.getString("txt"); $$;

The implicitly available udfContext field (or binding for script UDFs) provides the necessary functionality to create new UDT and tuple values:

CREATE TYPE custom_type (txt text, i int);
CREATE FUNCTION fct\_using\_udt ( somearg int )
    RETURNS NULL ON NULL INPUT
    RETURNS custom_type
    LANGUAGE java
    AS $$
        UDTValue udt = udfContext.newReturnUDTValue();
        udt.setString("txt", "some string");
        udt.setInt("i", 42);
        return udt;
    $$;

The definition of the UDFContext interface can be found in the Apache Cassandra source code for org.apache.cassandra.cql3.functions.UDFContext.

public interface UDFContext
{
    UDTValue newArgUDTValue(String argName);
    UDTValue newArgUDTValue(int argNum);
    UDTValue newReturnUDTValue();
    UDTValue newUDTValue(String udtName);
    TupleValue newArgTupleValue(String argName);
    TupleValue newArgTupleValue(int argNum);
    TupleValue newReturnTupleValue();
    TupleValue newTupleValue(String cqlDefinition);
}

Java UDFs already have some imports for common interfaces and classes defined. These imports are:

import java.nio.ByteBuffer;
import java.util.List;
import java.util.Map;
import java.util.Set;
import org.apache.cassandra.cql3.functions.UDFContext;
import com.datastax.driver.core.TypeCodec;
import com.datastax.driver.core.TupleValue;
import com.datastax.driver.core.UDTValue;

Please note, that these convenience imports are not available for script UDFs.

CREATE FUNCTION statement

Creating a new user-defined function uses the CREATE FUNCTION statement:

create_function_statement::= CREATE [ OR REPLACE ] FUNCTION [ IF NOT EXISTS]
	function_name '(' arguments_declaration ')'
	[ CALLED | RETURNS NULL ] ON NULL INPUT
	RETURNS cql_type
	LANGUAGE identifier
	AS string arguments_declaration: identifier cql_type ( ',' identifier cql_type )*

For example:

CREATE OR REPLACE FUNCTION somefunction(somearg int, anotherarg text, complexarg frozen<someUDT>, listarg list)
    RETURNS NULL ON NULL INPUT
    RETURNS text
    LANGUAGE java
    AS $$
        // some Java code
    $$;

CREATE FUNCTION IF NOT EXISTS akeyspace.fname(someArg int)
    CALLED ON NULL INPUT
    RETURNS text
    LANGUAGE java
    AS $$
        // some Java code
    $$;

CREATE FUNCTION with the optional OR REPLACE keywords creates either a function or replaces an existing one with the same signature. A CREATE FUNCTION without OR REPLACE fails if a function with the same signature already exists. If the optional IF NOT EXISTS keywords are used, the function will only be created only if another function with the same signature does not exist. OR REPLACE and IF NOT EXISTS cannot be used together.

Behavior for null input values must be defined for each function:

  • RETURNS NULL ON NULL INPUT declares that the function will always return null if any of the input arguments is null.

  • CALLED ON NULL INPUT declares that the function will always be executed.

Function Signature

Signatures are used to distinguish individual functions. The signature consists of a fully-qualified function name of the <keyspace>.<function_name> and a concatenated list of all the argument types.

Note that keyspace names, function names and argument types are subject to the default naming conventions and case-sensitivity rules.

Functions belong to a keyspace; if no keyspace is specified, the current keyspace is used. User-defined functions are not allowed in the system keyspaces.

DROP FUNCTION statement

Dropping a function uses the DROP FUNCTION statement:

drop_function_statement::= DROP FUNCTION [ IF EXISTS ] function_name [ '(' arguments_signature ')' ]
arguments_signature::= cql_type ( ',' cql_type )*

For example:

DROP FUNCTION myfunction;
DROP FUNCTION mykeyspace.afunction;
DROP FUNCTION afunction ( int );
DROP FUNCTION afunction ( text );

You must specify the argument types of the function, the arguments_signature, in the drop command if there are multiple overloaded functions with the same name but different signatures. DROP FUNCTION with the optional IF EXISTS keywords drops a function if it exists, but does not throw an error if it doesn’t.

Aggregate functions

Aggregate functions work on a set of rows. Values for each row are input, to return a single value for the set of rows aggregated.

If normal columns, scalar functions, UDT fields, writetime, or ttl are selected together with aggregate functions, the values returned for them will be the ones of the first row matching the query.

Native aggregates

Count

The count function can be used to count the rows returned by a query.

For example:

SELECT COUNT (*) FROM plays;
SELECT COUNT (1) FROM plays;

It also can count the non-null values of a given column:

SELECT COUNT (scores) FROM plays;

Max and Min

The max and min functions compute the maximum and the minimum value returned by a query for a given column.

For example:

SELECT MIN (players), MAX (players) FROM plays WHERE game = 'quake';

Sum

The sum function sums up all the values returned by a query for a given column.

The returned value is of the same type as the input collection elements, so there is a risk of overflowing if the sum of the values exceeds the maximum value that the type can represent.

For example:

SELECT SUM (players) FROM plays;

The returned value is of the same type as the input values, so there is a risk of overflowing the type if the sum of the values exceeds the maximum value that the type can represent. You can use type casting to cast the input values as a type large enough to contain the type. For example:

SELECT SUM (CAST (players AS VARINT)) FROM plays;

Avg

The avg function computes the average of all the values returned by a query for a given column.

For example:

SELECT AVG (players) FROM plays;

The average of an empty collection returns zero.

The returned value is of the same type as the input values, which might include rounding and truncations. For example collection_avg([1, 2]) returns 1 instead of 1.5. You can use type casting to cast to a type with the desired decimal precision. For example:

SELECT AVG (CAST (players AS FLOAT)) FROM plays;

User-Defined Aggregates (UDAs)

User-defined aggregates allow the creation of custom aggregate functions. User-defined aggregates can be used in SELECT statement.

Each aggregate requires an initial state of type STYPE defined with the INITCOND`value (default value: `null). The first argument of the state function must have type STYPE. The remaining arguments of the state function must match the types of the user-defined aggregate arguments. The state function is called once for each row, and the value returned by the state function becomes the new state. After all rows are processed, the optional FINALFUNC is executed with last state value as its argument.

The STYPE value is mandatory in order to distinguish possibly overloaded versions of the state and/or final function, since the overload can appear after creation of the aggregate.

A complete working example for user-defined aggregates (assuming that a keyspace has been selected using the USE statement):

CREATE OR REPLACE FUNCTION test.averageState(state tuple<int,bigint>, val int)
    CALLED ON NULL INPUT
    RETURNS tuple
    LANGUAGE java
    AS $$
        if (val != null) {
            state.setInt(0, state.getInt(0)+1);
            state.setLong(1, state.getLong(1)+val.intValue());
        }
        return state;
    $$;

CREATE OR REPLACE FUNCTION test.averageFinal (state tuple<int,bigint>)
    CALLED ON NULL INPUT
    RETURNS double
    LANGUAGE java
    AS $$
        double r = 0;
        if (state.getInt(0) == 0) return null;
        r = state.getLong(1);
        r /= state.getInt(0);
        return Double.valueOf(r);
    $$;

CREATE OR REPLACE AGGREGATE test.average(int)
    SFUNC averageState
    STYPE tuple
    FINALFUNC averageFinal
    INITCOND (0, 0);

CREATE TABLE test.atable (
    pk int PRIMARY KEY,
    val int
);

INSERT INTO test.atable (pk, val) VALUES (1,1);
INSERT INTO test.atable (pk, val) VALUES (2,2);
INSERT INTO test.atable (pk, val) VALUES (3,3);
INSERT INTO test.atable (pk, val) VALUES (4,4);

SELECT test.average(val) FROM atable;

CREATE AGGREGATE statement

Creating (or replacing) a user-defined aggregate function uses the CREATE AGGREGATE statement:

create_aggregate_statement ::= CREATE [ OR REPLACE ] AGGREGATE [ IF NOT EXISTS ]
                                function_name '(' arguments_signature')'
                                SFUNC function_name
                                STYPE cql_type:
                                [ FINALFUNC function_name]
                                [ INITCOND term ]

See above for a complete example.

The CREATE AGGREGATE command with the optional OR REPLACE keywords creates either an aggregate or replaces an existing one with the same signature. A CREATE AGGREGATE without OR REPLACE fails if an aggregate with the same signature already exists. The CREATE AGGREGATE command with the optional IF NOT EXISTS keywords creates an aggregate if it does not already exist. The OR REPLACE and IF NOT EXISTS phrases cannot be used together.

The STYPE value defines the type of the state value and must be specified. The optional INITCOND defines the initial state value for the aggregate; the default value is null. A non-null INITCOND must be specified for state functions that are declared with RETURNS NULL ON NULL INPUT.

The SFUNC value references an existing function to use as the state-modifying function. The first argument of the state function must have type STYPE. The remaining arguments of the state function must match the types of the user-defined aggregate arguments. The state function is called once for each row, and the value returned by the state function becomes the new state. State is not updated for state functions declared with RETURNS NULL ON NULL INPUT and called with null. After all rows are processed, the optional FINALFUNC is executed with last state value as its argument. It must take only one argument with type STYPE, but the return type of the FINALFUNC may be a different type. A final function declared with RETURNS NULL ON NULL INPUT means that the aggregate’s return value will be null, if the last state is null.

If no FINALFUNC is defined, the overall return type of the aggregate function is STYPE. If a FINALFUNC is defined, it is the return type of that function.

DROP AGGREGATE statement

Dropping an user-defined aggregate function uses the DROP AGGREGATE statement:

drop_aggregate_statement::= DROP AGGREGATE [ IF EXISTS ] function_name[ '(' arguments_signature ')'
]

For instance:

DROP AGGREGATE myAggregate;
DROP AGGREGATE myKeyspace.anAggregate;
DROP AGGREGATE someAggregate ( int );
DROP AGGREGATE someAggregate ( text );

The DROP AGGREGATE statement removes an aggregate created using CREATE AGGREGATE. You must specify the argument types of the aggregate to drop if there are multiple overloaded aggregates with the same name but a different signature.

The DROP AGGREGATE command with the optional IF EXISTS keywords drops an aggregate if it exists, and does nothing if a function with the signature does not exist.