Add allow_experimental_funnel_functions and fix argument order of sequenceNextNode

This commit is contained in:
achimbab 2021-04-08 00:26:51 +09:00
parent 63dc35780e
commit d13d69e246
4 changed files with 124 additions and 110 deletions

View File

@ -5,6 +5,8 @@
#include <DataTypes/DataTypeDate.h>
#include <DataTypes/DataTypeDateTime.h>
#include <DataTypes/DataTypeNullable.h>
#include <Interpreters/Context.h>
#include <Common/CurrentThread.h>
#include <ext/range.h>
@ -20,6 +22,7 @@ namespace ErrorCodes
extern const int ILLEGAL_TYPE_OF_ARGUMENT;
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
extern const int BAD_ARGUMENTS;
extern const int FUNCTION_NOT_ALLOWED;
}
namespace
@ -85,8 +88,8 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events,
"Aggregate function '{}' requires at most {} (timestamp, value_column, ...{} events) arguments.",
name, max_events + min_required_args, max_events), ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
if (const auto * cond_arg = argument_types[1].get(); cond_arg && !isUInt8(cond_arg))
throw Exception("Illegal type " + cond_arg->getName() + " of argument 1 of aggregate function "
if (const auto * cond_arg = argument_types[2].get(); cond_arg && !isUInt8(cond_arg))
throw Exception("Illegal type " + cond_arg->getName() + " of third argument of aggregate function "
+ name + ", must be UInt8", ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
for (const auto i : ext::range(min_required_args, argument_types.size()))
@ -98,12 +101,12 @@ createAggregateFunctionSequenceNode(const std::string & name, UInt64 max_events,
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT);
}
if (WhichDataType(argument_types[2].get()).idx != TypeIndex::String)
if (WhichDataType(argument_types[1].get()).idx != TypeIndex::String)
throw Exception{"Illegal type " + argument_types[1].get()->getName()
+ " of second argument of aggregate function " + name + ", must be String",
ErrorCodes::ILLEGAL_TYPE_OF_ARGUMENT};
DataTypePtr data_type = makeNullable(argument_types[2]);
DataTypePtr data_type = makeNullable(argument_types[1]);
WhichDataType timestamp_type(argument_types[0].get());
if (timestamp_type.idx == TypeIndex::UInt8)
@ -128,6 +131,14 @@ auto createAggregateFunctionSequenceNodeMaxArgs(UInt64 max_events)
{
return [max_events](const std::string & name, const DataTypes & argument_types, const Array & parameters)
{
if (CurrentThread::isInitialized())
{
const Context * query_context = CurrentThread::get().getQueryContext();
if (query_context && !query_context->getSettingsRef().allow_experimental_funnel_functions)
throw Exception{"Cannot call 'sequenceNextNode' aggregate function because experimental_funnel_functions is not allowed. "
"Set 'allow_experimental_funnel_functions = 1' setting to enable", ErrorCodes::FUNCTION_NOT_ALLOWED};
}
return createAggregateFunctionSequenceNode(name, max_events, argument_types, parameters);
};
}

View File

@ -160,8 +160,8 @@ class SequenceNextNodeImpl final
static Data & data(AggregateDataPtr place) { return *reinterpret_cast<Data *>(place); }
static const Data & data(ConstAggregateDataPtr place) { return *reinterpret_cast<const Data *>(place); }
static constexpr size_t base_cond_column_idx = 1;
static constexpr size_t event_column_idx = 2;
static constexpr size_t base_cond_column_idx = 2;
static constexpr size_t event_column_idx = 1;
SequenceBase seq_base_kind;
SequenceDirection seq_direction;

View File

@ -428,6 +428,7 @@ class IColumn;
M(Bool, optimize_on_insert, true, "Do the same transformation for inserted block of data as if merge was done on this block.", 0) \
M(Bool, allow_experimental_map_type, false, "Allow data type Map", 0) \
M(Bool, allow_experimental_window_functions, false, "Allow experimental window functions", 0) \
M(Bool, allow_experimental_funnel_functions, false, "Allow sequenceNextNode aggregate funcion", 0) \
M(Bool, use_antlr_parser, false, "Parse incoming queries using ANTLR-generated experimental parser", 0) \
M(Bool, async_socket_for_remote, true, "Asynchronously read from socket executing remote query", 0) \
\

View File

@ -1,3 +1,5 @@
SET allow_experimental_funnel_functions = 1;
DROP TABLE IF EXISTS test_sequenceNextNode_Nullable;
CREATE TABLE IF NOT EXISTS test_sequenceNextNode_Nullable (dt DateTime, id int, action Nullable(String)) ENGINE = MergeTree() PARTITION BY dt ORDER BY id;
@ -27,26 +29,26 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',6,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',6,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:05',6,'C');
SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',10,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,'B');
@ -54,41 +56,41 @@ INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:02',10,NULL
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:03',10,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:04',10,'D');
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id >= 10 GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'A');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'B');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'C');
INSERT INTO test_sequenceNextNode_Nullable values ('1970-01-01 09:00:01',11,'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL);
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A');
SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null);
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is NULL);
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node = 'A');
SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable WHERE id = 11 GROUP BY id HAVING next_node is null);
SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode_Nullable WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode_Nullable GROUP BY id ORDER BY id;
DROP TABLE IF EXISTS test_sequenceNextNode_Nullable;
@ -123,74 +125,74 @@ INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',6,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',6,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:05',6,'C');
SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, D)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, E)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, D)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, E)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'E') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A->C)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->C)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, A->A->B)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, head, B->A->A)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, A->A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A', action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, tail, B->A->A)', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'B', action = 'A', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',10,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:02',10,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:03',10,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:04',10,'D');
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(forward, head, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail, A) id >= 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'D', action = 'C', action = 'B') AS next_node FROM test_sequenceNextNode WHERE id >= 10 GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'B');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'C');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',11,'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL);
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A');
SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, 1, action, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null);
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'B', action ='C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A->B->C) id = 11', count() FROM (SELECT id, sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'D');
SELECT '(0, A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is NULL);
SELECT '(0, C) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'B');
SELECT '(0, C->B) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action ='B') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node = 'A');
SELECT '(0, C->B->A) id = 11', count() FROM (SELECT id, sequenceNextNode('backward', 'tail')(dt, action, 1, action = 'C', action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 11 GROUP BY id HAVING next_node is null);
SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, 1, action) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(forward, head) id < 10', id, sequenceNextNode('forward', 'head')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(backward, tail) id < 10', id, sequenceNextNode('backward', 'tail')(dt, action, 1) AS next_node FROM test_sequenceNextNode WHERE id < 10 GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, A->B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'A', action = 'B', action = 'C') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->B)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(forward, first_match, B->A)', id, sequenceNextNode('forward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, 1, action, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->A)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'A') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(backward, first_match, B->B)', id, sequenceNextNode('backward', 'first_match')(dt, action, 1, action = 'B', action = 'B') AS next_node FROM test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, 1, action, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id;
SELECT '(max_args)', id, sequenceNextNode('forward', 'head')(dt, action, 1, action = '0', action = '1', action = '2', action = '3', action = '4', action = '5', action = '6', action = '7', action = '8', action = '9', action = '10', action = '11', action = '12', action = '13', action = '14', action = '15', action = '16', action = '17', action = '18', action = '19', action = '20', action = '21', action = '22', action = '23', action = '24', action = '25', action = '26', action = '27', action = '28', action = '29', action = '30', action = '31', action = '32', action = '33', action = '34', action = '35', action = '36', action = '37', action = '38', action = '39', action = '40', action = '41', action = '42', action = '43', action = '44', action = '45', action = '46', action = '47', action = '48', action = '49', action = '50', action = '51', action = '52', action = '53', action = '54', action = '55', action = '56', action = '57', action = '58', action = '59', action = '60', action = '61', action = '62', action = '63') from test_sequenceNextNode GROUP BY id ORDER BY id;
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A');
INSERT INTO test_sequenceNextNode values ('1970-01-01 09:00:01',12,'A');
SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, 1, action, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12;
SELECT '(forward, head, A) id = 12', sequenceNextNode('forward', 'head')(dt, action, 1, action = 'A') AS next_node FROM test_sequenceNextNode WHERE id = 12;
DROP TABLE IF EXISTS test_sequenceNextNode;
@ -211,18 +213,18 @@ INSERT INTO test_base_condition values ('1970-01-01 09:00:02',3,'B','2');
INSERT INTO test_base_condition values ('1970-01-01 09:00:03',3,'D','3');
INSERT INTO test_base_condition values ('1970-01-01 09:00:04',3,'C','4');
SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, head, 1)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, head, 1, A)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, head, 1, A->B)', id, sequenceNextNode('forward', 'head')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action) AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, referrer = '1', action, action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, tail, 1)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, tail, 1, A)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, tail, 1, A->B)', id, sequenceNextNode('backward', 'tail')(dt, action, referrer = '1', action = 'A', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, first_match, 1, B)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(forward, first_match, 1, B->C)', id, sequenceNextNode('forward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'C') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, referrer = '2', action, action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, first_match, 1, B)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
SELECT '(backward, first_match, 1, B->C)', id, sequenceNextNode('backward', 'first_match')(dt, action, referrer = '2', action = 'B', action = 'A') AS next_node FROM test_base_condition GROUP BY id ORDER BY id;
DROP TABLE IF EXISTS test_base_condition;