diff --git a/src/backend/distributed/Makefile b/src/backend/distributed/Makefile index d2f441250..e7d908023 100644 --- a/src/backend/distributed/Makefile +++ b/src/backend/distributed/Makefile @@ -18,7 +18,7 @@ generated_downgrade_sql_files += $(patsubst %,$(citus_abs_srcdir)/build/sql/%,$( DATA_built = $(generated_sql_files) # directories with source files -SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib test transaction utils worker +SUBDIRS = . commands connection ddl deparser executor metadata operations planner progress relay safeclib test transaction utils worker cimv # Symlinks are not copied over to the build directory if a separete build # directory is used during configure (such as on CI) diff --git a/src/backend/distributed/cimv/create.c b/src/backend/distributed/cimv/create.c new file mode 100644 index 000000000..6ec008aa7 --- /dev/null +++ b/src/backend/distributed/cimv/create.c @@ -0,0 +1,1605 @@ +#include "postgres.h" +#include "access/htup_details.h" +#include "access/reloptions.h" +#include "access/xact.h" +#include "catalog/heap.h" +#include "catalog/index.h" +#include "catalog/pg_aggregate.h" +#include "catalog/pg_class.h" +#include "catalog/pg_operator.h" +#include "catalog/pg_proc.h" +#include "catalog/pg_type.h" +#include "catalog/toasting.h" +#include "commands/defrem.h" +#include "commands/tablecmds.h" +#include "commands/view.h" +#include "distributed/citus_ruleutils.h" +#include "distributed/pg_cimv.h" +#include "distributed/listutils.h" +#include "distributed/metadata_cache.h" +#include "executor/spi.h" +#include "miscadmin.h" +#include "nodes/makefuncs.h" +#include "nodes/nodeFuncs.h" +#include "optimizer/clauses.h" +#include "parser/analyze.h" +#include "parser/parser.h" +#include "utils/builtins.h" +#include "utils/fmgroids.h" +#include "utils/lsyscache.h" +#include "utils/syscache.h" + +#include "distributed/cimv.h" + +#define PARTIAL_AGG_FNNAME "partial_agg" +#define COMBINE_AGG_FNNAME "combine_agg" +#define FINALIZE_AGG_FNNAME "finalize_agg" + +typedef struct +{ + bool isCimv; + bool insertOnlyCapture; + char *schedule; + List *storageOptions; +} MatViewCreateOptions; + +typedef struct +{ + const CreateTableAsStmt *stmt; + MatViewCreateOptions *createOptions; + Form_pg_cimv formCimv; + RangeVar *baseTableName; + RangeVar *matTableName; + RangeVar *landingTableName; + RangeVar *userViewName; + RangeVar *refreshViewName; + RangeVar *baseTableNameQuoted; + RangeVar *matTableNameQuoted; + RangeVar *landingTableNameQuoted; + RangeVar *userViewNameQuoted; + RangeVar *refreshViewNameQuoted; + List *targetListEntries; + List *groupTargetListEntries; + List *aggTargetListEntries; + CitusTableCacheEntry *citusTable; + TargetEntry *partitionColumn; + bool supportsDelete; +} CimvCreate; + +static void CreateCimv(CimvCreate *cimvCreate); +static void CreateMatTable(CimvCreate *cimvCreate, bool isLandingZone); +static void CreateIndexOnMatTable(CimvCreate *cimvCreate); +static void DistributeTable(CimvCreate *cimvCreate, RangeVar *tableName); +static void CreateUserView(CimvCreate *cimvCreate); +static void CreateRefreshView(CimvCreate *cimvCreate); +static void CreateDataChangeTriggerFunction(CimvCreate *cimvCreate); +static void CreateCronJob(CimvCreate *cimvCreate); +static void DataChangeTriggerFunctionAppendInsertDelete(CimvCreate *cimvCreate, StringInfo + buf, bool isInsert); +static void DataChangeTriggerFunctionAppendErrorOnDelete(CimvCreate *cimvCreate, + StringInfo buf); +static void AppendOnConflict(CimvCreate *cimvCreate, StringInfo buf, bool isInsert); +static void CreateDataChangeTriggers(CimvCreate *cimvCreate); +static void CreateDataChangeTrigger(CimvCreate *cimvCreate, int triggerEvent); +static bool ValidateCimv(CimvCreate *cimvCreate); + +static void ValidateAgg(Aggref *agg, bool *supportsDelete); +static void AddCountAgg(Query *query, bool isInsert); +static Node * PartializeAggs(Node *node, void *context); +static CimvCreate * InitializeCimvCreate(const CreateTableAsStmt *stmt, + MatViewCreateOptions *createOptions); +static MatViewCreateOptions * GetMatViewCreateOptions(const CreateTableAsStmt *stmt); + +static ObjectAddress DefineVirtualRelation(RangeVar *relation, List *tlist, + Query *viewParse); +static Oid CitusFunctionOidWithSignature(char *functionName, int numargs, Oid *argtypes); +static Oid PartialAggOid(void); +static void AppendQuotedLiteral(StringInfo buf, const char *val); +static void AppendStringInfoFunction(StringInfo buf, Oid fnoid); +static Oid AggregateFunctionOid(const char *functionName, Oid inputType); + +bool +ProcessCreateMaterializedViewStmt(const CreateTableAsStmt *stmt, const char *query_string, + PlannedStmt *pstmt) +{ + bool stmtHandled = false; + CimvCreate *cimvCreate = NULL; + + if (stmt->relkind != OBJECT_MATVIEW) + { + return stmtHandled; + } + + MatViewCreateOptions *options = GetMatViewCreateOptions(stmt); + + if (options->isCimv) + { + cimvCreate = InitializeCimvCreate(stmt, options); + + ValidateCimv(cimvCreate); + CreateCimv(cimvCreate); + stmtHandled = true; + } + + if (options != NULL) + { + pfree(options); + } + if (cimvCreate != NULL) + { + pfree(cimvCreate); + } + + return stmtHandled; +} + + +static void +CreateCimv(CimvCreate *cimvCreate) +{ + if (SPI_connect() != SPI_OK_CONNECT) + { + elog(ERROR, "SPI_connect failed"); + } + + CreateMatTable(cimvCreate, false); + + if (cimvCreate->createOptions->schedule != NULL) + { + CreateMatTable(cimvCreate, true); + CreateCronJob(cimvCreate); + } + + CreateUserView(cimvCreate); + CreateRefreshView(cimvCreate); + CreateDataChangeTriggerFunction(cimvCreate); + CreateDataChangeTriggers(cimvCreate); + InsertIntoPgCimv(cimvCreate->formCimv); + + if (SPI_finish() != SPI_OK_FINISH) + { + elog(ERROR, "SPI_finish failed"); + } + + if (!cimvCreate->stmt->into->skipData) + { + RefreshCimv(cimvCreate->formCimv, cimvCreate->stmt->into->skipData, true); + } +} + + +static void +CreateMatTable(CimvCreate *cimvCreate, bool isLandingZone) +{ + RangeVar *table = isLandingZone ? cimvCreate->landingTableName : + cimvCreate->matTableName; + + /* postgres/src/backend/commands/createas.c create_ctas_internal*/ + Datum toast_options; + static char *validnsps[] = HEAP_RELOPT_NAMESPACES; + + /* + * Create the target relation by faking up a CREATE TABLE parsetree and + * passing it to DefineRelation. + */ + CreateStmt *create = makeNode(CreateStmt); + create->relation = table; + create->tableElts = NIL; + create->inhRelations = NIL; + create->ofTypename = NULL; + create->constraints = NIL; + create->options = cimvCreate->createOptions->storageOptions; + create->oncommit = ONCOMMIT_NOOP; + create->tablespacename = cimvCreate->stmt->into->tableSpaceName; + create->if_not_exists = false; +#if PG_VERSION_NUM >= PG_VERSION_12 + create->accessMethod = cimvCreate->stmt->into->accessMethod; +#endif + + ColumnDef *col; + TargetEntry *tle; + foreach_ptr(tle, cimvCreate->targetListEntries) + { + if (IsA(tle->expr, Aggref)) + { + col = makeColumnDef(tle->resname, + BYTEAOID, + -1, + InvalidOid); + create->tableElts = lappend(create->tableElts, col); + } + else + { + col = makeColumnDef(tle->resname, + exprType((Node *) tle->expr), + exprTypmod((Node *) tle->expr), + exprCollation((Node *) tle->expr)); + create->tableElts = lappend(create->tableElts, col); + } + } + + col = makeColumnDef("__count__", + INT8OID, + -1, + InvalidOid); + create->tableElts = lappend(create->tableElts, col); + + /* + * Create the relation. (This will error out if there's an existing view, + * so we don't need more code to complain if "replace" is false.) + */ + Oid tableOid = DefineRelation(create, RELKIND_RELATION, GetUserId(), NULL, + NULL).objectId; + + /* + * If necessary, create a TOAST table for the target table. Note that + * NewRelationCreateToastTable ends with CommandCounterIncrement(), so + * that the TOAST table will be visible for insertion. + */ + CommandCounterIncrement(); + + /* parse and validate reloptions for the toast table */ + toast_options = transformRelOptions((Datum) 0, + create->options, + "toast", + validnsps, + true, false); + + (void) heap_reloptions(RELKIND_TOASTVALUE, toast_options, true); + + NewRelationCreateToastTable(tableOid, toast_options); + + if (isLandingZone) + { + cimvCreate->formCimv->landingtable = tableOid; + } + else + { + cimvCreate->formCimv->mattable = tableOid; + CreateIndexOnMatTable(cimvCreate); + } + + if (cimvCreate->citusTable != NULL) + { + DistributeTable(cimvCreate, table); + } +} + + +static void +CreateIndexOnMatTable(CimvCreate *cimvCreate) +{ + StringInfo indexName = makeStringInfo(); + appendStringInfo(indexName, "%s_uidx", cimvCreate->matTableName->relname); + + IndexStmt *indexcreate = makeNode(IndexStmt); + indexcreate->isconstraint = true; + indexcreate->unique = true; + indexcreate->accessMethod = DEFAULT_INDEX_TYPE; + indexcreate->idxname = indexName->data; + indexcreate->relation = cimvCreate->matTableName; + indexcreate->tableSpace = cimvCreate->stmt->into->tableSpaceName; + indexcreate->indexParams = NIL; + indexcreate->indexIncludingParams = NIL; + + TargetEntry *tle; + foreach_ptr(tle, cimvCreate->groupTargetListEntries) + { + IndexElem *iparam = makeNode(IndexElem); + iparam->name = tle->resname; + iparam->indexcolname = NULL; + iparam->collation = NIL; + iparam->opclass = NIL; + iparam->ordering = SORTBY_DEFAULT; + iparam->nulls_ordering = SORTBY_NULLS_DEFAULT; + indexcreate->indexParams = lappend(indexcreate->indexParams, iparam); + } + + DefineIndex(cimvCreate->formCimv->mattable, + indexcreate, + InvalidOid, /* indexRelationId */ + InvalidOid, /* parentIndexId */ + InvalidOid, /* parentConstraintId */ + false, /* is_alter_table */ + false, /* check_rights */ + false, /* check_not_in_use */ + false, /* skip_build */ + false); /* quiet */ + + IndexElem *ie; + foreach_ptr(ie, indexcreate->indexParams) + { + pfree(ie); + } + pfree(indexcreate); + pfree(indexName->data); +} + + +static void +DistributeTable(CimvCreate *cimvCreate, RangeVar *tableName) +{ + StringInfoData querybuf; + initStringInfo(&querybuf); + + appendStringInfo(&querybuf, + "SELECT * FROM create_distributed_table($param$%s.%s$param$, $param$%s$param$, colocate_with => $param$%s.%s$param$);", + tableName->schemaname, + tableName->relname, + cimvCreate->partitionColumn->resname, + cimvCreate->baseTableName->schemaname, + cimvCreate->baseTableName->relname); + + if (SPI_execute(querybuf.data, false, 0) != SPI_OK_SELECT) + { + elog(ERROR, "SPI_exec failed: %s", querybuf.data); + } + + pfree(querybuf.data); +} + + +static void +CreateUserView(CimvCreate *cimvCreate) +{ + StringInfoData queryText; + initStringInfo(&queryText); + + bool unionWithLandingTable = cimvCreate->createOptions->schedule != NULL; + + appendStringInfoString(&queryText, "SELECT "); + + bool isFirst = true; + TargetEntry *tle; + foreach_ptr(tle, cimvCreate->targetListEntries) + { + if (isFirst) + { + isFirst = false; + } + else + { + appendStringInfoString(&queryText, ", "); + } + + const char *colName = quote_identifier(tle->resname); + if (IsA(tle->expr, Aggref)) + { + Aggref *aggref = castNode(Aggref, tle->expr); + appendStringInfo(&queryText, "%s(", FINALIZE_AGG_FNNAME); + AppendStringInfoFunction(&queryText, aggref->aggfnoid); + appendStringInfo(&queryText, ", %s, null::%s) AS %s", + colName, + format_type_with_typemod(aggref->aggtype, exprTypmod( + (Node *) aggref)), + colName); + } + else + { + appendStringInfo(&queryText, "%s", colName); + } + } + + if (unionWithLandingTable) + { + appendStringInfo(&queryText, + " FROM (SELECT * FROM %s.%s UNION ALL SELECT * FROM %s.%s) __union__", + cimvCreate->matTableNameQuoted->schemaname, + cimvCreate->matTableNameQuoted->relname, + cimvCreate->landingTableNameQuoted->schemaname, + cimvCreate->landingTableNameQuoted->relname); + } + else + { + appendStringInfo(&queryText, " FROM %s.%s", + cimvCreate->matTableNameQuoted->schemaname, + cimvCreate->matTableNameQuoted->relname); + } + + appendStringInfoString(&queryText, " GROUP BY "); + + isFirst = true; + foreach_ptr(tle, cimvCreate->groupTargetListEntries) + { + if (isFirst) + { + isFirst = false; + } + else + { + appendStringInfoString(&queryText, ", "); + } + appendStringInfo(&queryText, "%s", quote_identifier(tle->resname)); + } + + appendStringInfoString(&queryText, " HAVING sum(__count__) > 0"); + + List *rawStmts = raw_parser(queryText.data); + Assert(list_length(rawStmts) == 1); + RawStmt *rawStmt = linitial(rawStmts); + Query *query = parse_analyze(rawStmt, queryText.data, InvalidOid, 0, NULL); + + cimvCreate->formCimv->userview = DefineVirtualRelation(cimvCreate->userViewName, + query->targetList, + query).objectId; + + pfree(queryText.data); +} + + +static void +CreateRefreshView(CimvCreate *cimvCreate) +{ + Query *query = (Query *) copyObject(cimvCreate->stmt->into->viewQuery); + int inverse = 0; + query->targetList = (List *) PartializeAggs((Node *) query->targetList, &inverse); + AddCountAgg(query, true); + + StringInfoData querybuf; + initStringInfo(&querybuf); + + pg_get_query_def(query, &querybuf); + List *rawStmts = raw_parser(querybuf.data); + Assert(list_length(rawStmts) == 1); + RawStmt *rawStmt = linitial(rawStmts); + pfree(query); + query = parse_analyze(rawStmt, querybuf.data, InvalidOid, 0, NULL); + cimvCreate->formCimv->refreshview = DefineVirtualRelation(cimvCreate->refreshViewName, + query->targetList, + query).objectId; + + pfree(query); + pfree(querybuf.data); +} + + +static void +CreateCronJob(CimvCreate *cimvCreate) +{ + StringInfoData queryText; + initStringInfo(&queryText); + + appendStringInfo(&queryText, + "SELECT cron.schedule($cron_schedule$%s$cron_schedule$, $cron_schedule$", + cimvCreate->createOptions->schedule); + + appendStringInfo(&queryText, "WITH __del__ AS (DELETE FROM %s.%s RETURNING *) ", + cimvCreate->landingTableNameQuoted->schemaname, + cimvCreate->landingTableNameQuoted->relname); + + appendStringInfo(&queryText, "INSERT INTO %s.%s AS __mat__ ", + cimvCreate->matTableNameQuoted->schemaname, + cimvCreate->matTableNameQuoted->relname); + + appendStringInfoString(&queryText, "SELECT "); + + bool isFirst = true; + TargetEntry *tle; + foreach_ptr(tle, cimvCreate->targetListEntries) + { + if (isFirst) + { + isFirst = false; + } + else + { + appendStringInfoString(&queryText, ", "); + } + + + const char *colname = quote_identifier(tle->resname); + if (IsA(tle->expr, Aggref)) + { + Aggref *aggref = castNode(Aggref, tle->expr); + appendStringInfo(&queryText, "%s(", COMBINE_AGG_FNNAME); + AppendStringInfoFunction(&queryText, aggref->aggfnoid); + appendStringInfo(&queryText, ", %s) AS %s", colname, colname); + } + else + { + appendStringInfo(&queryText, "%s", colname); + } + } + + appendStringInfo(&queryText, ", sum(__count__) AS __count__ FROM __del__ GROUP BY "); + + isFirst = true; + foreach_ptr(tle, cimvCreate->groupTargetListEntries) + { + if (isFirst) + { + isFirst = false; + } + else + { + appendStringInfoString(&queryText, ", "); + } + appendStringInfo(&queryText, "%s", quote_identifier(tle->resname)); + } + + AppendOnConflict(cimvCreate, &queryText, true); + + appendStringInfoString(&queryText, ";$cron_schedule$);"); /* TODO: delete where count = 0 */ + + /* CREATE */ + if (SPI_execute(queryText.data, false, 0) != SPI_OK_SELECT) + { + elog(ERROR, "SPI_exec failed: %s", queryText.data); + } + + if (SPI_processed != 1 || SPI_tuptable->tupdesc->natts != 1) + { + elog(ERROR, "Error creating schedule"); + } + + bool isNull; + cimvCreate->formCimv->jobid = + DatumGetInt64(SPI_getbinval(SPI_tuptable->vals[0], + SPI_tuptable->tupdesc, + 1, + &isNull)); + + if (isNull || cimvCreate->formCimv->jobid < 1) + { + elog(ERROR, "Error creating schedule"); + } + + pfree(queryText.data); +} + + +static void +CreateDataChangeTriggerFunction(CimvCreate *cimvCreate) +{ + StringInfoData buf; + initStringInfo(&buf); + + bool isCitusTable = cimvCreate->citusTable != NULL; + + if (isCitusTable) + { + appendStringInfo(&buf, + "SELECT * FROM run_command_on_workers($cmd$CREATE FUNCTION %s.%s() RETURNS trigger AS $$ BEGIN RETURN null; END; $$ LANGUAGE plpgsql$cmd$)", + quote_identifier(NameStr( + cimvCreate->formCimv->triggerfnnamespace)), + quote_identifier(NameStr(cimvCreate->formCimv->triggerfnname))); + + if (SPI_execute(buf.data, false, 0) != SPI_OK_SELECT) + { + elog(ERROR, "SPI_exec failed: %s", buf.data); + } + + resetStringInfo(&buf); + } + + appendStringInfo(&buf, + "CREATE OR REPLACE FUNCTION %s.%s() RETURNS TRIGGER AS $trigger_function$ BEGIN\n", + quote_identifier(NameStr(cimvCreate->formCimv->triggerfnnamespace)), + quote_identifier(NameStr(cimvCreate->formCimv->triggerfnname))); + + /* INSERT */ + appendStringInfoString(&buf, + "IF (TG_OP = $inside_trigger_function$INSERT$inside_trigger_function$ OR TG_OP = $inside_trigger_function$UPDATE$inside_trigger_function$) THEN\n"); + DataChangeTriggerFunctionAppendInsertDelete(cimvCreate, &buf, true); + appendStringInfoString(&buf, "END IF;\n"); + + /* DELETE */ + appendStringInfoString(&buf, + "IF (TG_OP = $inside_trigger_function$DELETE$inside_trigger_function$ OR TG_OP = $inside_trigger_function$UPDATE$inside_trigger_function$) THEN\n"); + if (cimvCreate->supportsDelete) + { + DataChangeTriggerFunctionAppendInsertDelete(cimvCreate, &buf, false); + } + else + { + DataChangeTriggerFunctionAppendErrorOnDelete(cimvCreate, &buf); + } + + appendStringInfoString(&buf, "END IF;\n"); + + /* TRUNCATE */ + appendStringInfoString(&buf, + "IF (TG_OP = $inside_trigger_function$TRUNCATE$inside_trigger_function$) THEN\n"); + + appendStringInfoString(&buf, + "EXECUTE format($exec_format$TRUNCATE TABLE %s; $exec_format$, TG_ARGV[0]);"); + + /* TODO: also truncate landing table if it exists */ + + appendStringInfoString(&buf, "END IF;\n"); + + appendStringInfoString(&buf, + "RETURN NULL; END; $trigger_function$ LANGUAGE plpgsql;"); + + /* CREATE */ + if (SPI_execute(buf.data, false, 0) != SPI_OK_UTILITY) + { + elog(ERROR, "SPI_exec failed: %s", buf.data); + } + + if (isCitusTable) + { + resetStringInfo(&buf); + appendStringInfo(&buf, + "SELECT FROM create_distributed_function($cdfn$%s.%s()$cdfn$)", + quote_identifier(NameStr( + cimvCreate->formCimv->triggerfnnamespace)), + quote_identifier(NameStr(cimvCreate->formCimv->triggerfnname))); + } + + int expectedResult = isCitusTable ? SPI_OK_SELECT : SPI_OK_UTILITY; + if (SPI_execute(buf.data, false, 0) != expectedResult) + { + elog(ERROR, "SPI_exec failed: %s", buf.data); + } + + pfree(buf.data); +} + + +static void +DataChangeTriggerFunctionAppendInsertDelete(CimvCreate *cimvCreate, StringInfo buf, bool + isInsert) +{ + StringInfoData querybuf; + initStringInfo(&querybuf); + + Query *query = (Query *) copyObject(cimvCreate->stmt->into->viewQuery); + int inverse = isInsert ? 0 : 1; + query->targetList = (List *) PartializeAggs((Node *) query->targetList, &inverse); + AddCountAgg(query, isInsert); + + RangeTblEntry *baseRte = (RangeTblEntry *) linitial(query->rtable); + baseRte->rtekind = RTE_CTE; + baseRte->ctename = isInsert ? "__ins__" : "__del__"; + + pg_get_query_def(query, &querybuf); + + appendStringInfoString(buf, "EXECUTE format($exec_format$INSERT INTO %s AS __mat__ "); + + /* SELECT */ + appendStringInfoString(buf, querybuf.data); + + /* ON CONFLICT */ + if (cimvCreate->createOptions->schedule == NULL) + { + AppendOnConflict(cimvCreate, buf, isInsert); + } + + appendStringInfoString(buf, ";\n$exec_format$, TG_ARGV[0]);"); + + if (!isInsert && cimvCreate->createOptions->schedule == NULL) + { + appendStringInfoString(buf, + "EXECUTE format($exec_format$DELETE FROM %s WHERE __count__ = 0;$exec_format$, TG_ARGV[0]);"); + } + + pfree(querybuf.data); + pfree(query); +} + + +static void +DataChangeTriggerFunctionAppendErrorOnDelete(CimvCreate *cimvCreate, StringInfo buf) +{ + appendStringInfo(buf, + "RAISE EXCEPTION $ex$MATERIALIZED VIEW '%s' on table '%s' does not support UPDATE/DELETE$ex$;\n", + cimvCreate->userViewNameQuoted->relname, + cimvCreate->baseTableNameQuoted->relname); +} + + +static void +AppendOnConflict(CimvCreate *cimvCreate, StringInfo buf, bool isInsert) +{ + appendStringInfoString(buf, " ON CONFLICT ("); + + bool isFirst = true; + TargetEntry *tle; + foreach_ptr(tle, cimvCreate->groupTargetListEntries) + { + if (!isFirst) + { + appendStringInfo(buf, ", %s", quote_identifier(tle->resname)); + } + else + { + appendStringInfo(buf, "%s", quote_identifier(tle->resname)); + isFirst = false; + } + } + + appendStringInfoString(buf, + ") DO UPDATE SET __count__ = __mat__.__count__ + EXCLUDED.__count__"); + + isFirst = true; + foreach_ptr(tle, cimvCreate->aggTargetListEntries) + { + Aggref *aggref = castNode(Aggref, tle->expr); + const char *colname = quote_identifier(tle->resname); + + appendStringInfo(buf, ", %s = (SELECT %s(", colname, COMBINE_AGG_FNNAME); + AppendStringInfoFunction(buf, aggref->aggfnoid); + appendStringInfo(buf, + ", __val__) FROM (VALUES (EXCLUDED.%s),(__mat__.%s)) AS __agg__(__val__))", + colname, + colname); + } +} + + +static void +CreateDataChangeTriggers(CimvCreate *cimvCreate) +{ + CreateDataChangeTrigger(cimvCreate, TRIGGER_EVENT_INSERT); + + if (!cimvCreate->createOptions->insertOnlyCapture) + { + CreateDataChangeTrigger(cimvCreate, TRIGGER_EVENT_DELETE); + CreateDataChangeTrigger(cimvCreate, TRIGGER_EVENT_UPDATE); + CreateDataChangeTrigger(cimvCreate, TRIGGER_EVENT_TRUNCATE); + } +} + + +static void +CreateDataChangeTrigger(CimvCreate *cimvCreate, int triggerEvent) +{ + StringInfoData buf; + initStringInfo(&buf); + + bool isCitusTable = cimvCreate->citusTable != NULL; + + char *event; + char *referencing; + switch (triggerEvent) + { + case TRIGGER_EVENT_INSERT: + event = "INSERT"; + referencing = "REFERENCING NEW TABLE AS __ins__ "; + break; + + case TRIGGER_EVENT_DELETE: + event = "DELETE"; + referencing = "REFERENCING OLD TABLE AS __del__ "; + break; + + case TRIGGER_EVENT_UPDATE: + event = "UPDATE"; + referencing = "REFERENCING NEW TABLE AS __ins__ OLD TABLE AS __del__ "; + break; + + case TRIGGER_EVENT_TRUNCATE: + event = "TRUNCATE"; + referencing = ""; + break; + + /* default: + * error; */ + } + + /* TODO: UPDATE [ OF column_name [, ... ] ] */ + StringInfoData triggerName; + initStringInfo(&triggerName); + appendStringInfo(&triggerName, "%s_%s", NameStr(cimvCreate->formCimv->triggerfnname), + event); + + RangeVar *insertTable = cimvCreate->createOptions->schedule == NULL ? + cimvCreate->matTableNameQuoted : + cimvCreate->landingTableNameQuoted; + + if (isCitusTable) + { + appendStringInfo(&buf, + "SELECT bool_and(success), max(result) FROM run_command_on_colocated_placements($param$%s.%s$param$, $param$%s.%s$param$, $create_trigger$ ", + cimvCreate->baseTableNameQuoted->schemaname, + cimvCreate->baseTableNameQuoted->relname, + insertTable->schemaname, + insertTable->relname); + + appendStringInfo(&buf, + "CREATE TRIGGER %s AFTER %s ON %%s %s " + "FOR EACH STATEMENT EXECUTE PROCEDURE %s.%s(%%L)", + quote_identifier(triggerName.data), + event, + referencing, + quote_identifier(NameStr( + cimvCreate->formCimv->triggerfnnamespace)), + quote_identifier(NameStr(cimvCreate->formCimv->triggerfnname))); + + appendStringInfoString(&buf, "$create_trigger$);"); + } + else + { + appendStringInfo(&buf, + "CREATE TRIGGER %s AFTER %s ON %s.%s %s " + "FOR EACH STATEMENT EXECUTE PROCEDURE %s.%s('%s.%s')", + quote_identifier(triggerName.data), + event, + cimvCreate->baseTableNameQuoted->schemaname, + cimvCreate->baseTableNameQuoted->relname, + referencing, + quote_identifier(NameStr( + cimvCreate->formCimv->triggerfnnamespace)), + quote_identifier(NameStr(cimvCreate->formCimv->triggerfnname)), + insertTable->schemaname, + insertTable->relname); + } + + int expectedResult = isCitusTable ? SPI_OK_SELECT : SPI_OK_UTILITY; + if (SPI_execute(buf.data, false, 0) != expectedResult) + { + elog(ERROR, "SPI_exec failed: %s", buf.data); + } + + if (isCitusTable) + { + if (SPI_tuptable != NULL && SPI_tuptable->tupdesc->natts == 2 && SPI_processed == + 1) + { + SPITupleTable *tuptable = SPI_tuptable; + TupleDesc tupdesc = tuptable->tupdesc; + HeapTuple tuple = tuptable->vals[0]; + + SPI_getvalue(tuple, tupdesc, 1); + bool isNull; + Datum isSuccessDatum = SPI_getbinval(tuple, tupdesc, 1, &isNull); + + if (!isNull && !DatumGetBool(isSuccessDatum)) + { + elog(ERROR, "SPI_exec failed: %s", SPI_getvalue(tuple, tupdesc, 2)); + } + } + } + + pfree(buf.data); +} + + +static CimvCreate * +InitializeCimvCreate(const CreateTableAsStmt *stmt, MatViewCreateOptions *createOptions) +{ + CimvCreate *cimvCreate = palloc(sizeof(CimvCreate)); + cimvCreate->formCimv = palloc(sizeof(FormData_pg_cimv)); + cimvCreate->formCimv->jobid = 0; + cimvCreate->formCimv->landingtable = InvalidOid; + + namestrcpy(&cimvCreate->formCimv->triggerfnnamespace, CITUS_INTERNAL_SCHEMA); + namestrcpy(&cimvCreate->formCimv->triggerfnname, stmt->into->rel->relname); + + StringInfo mat = makeStringInfo(); + appendStringInfo(mat, "%s_cimv_%s", stmt->into->rel->relname, + MATERIALIZATION_TABLE_SUFFIX); + + StringInfo rv = makeStringInfo(); + appendStringInfo(rv, "%s_cimv_%s", stmt->into->rel->relname, REFRESH_VIEW_SUFFIX); + + StringInfo ld = makeStringInfo(); + appendStringInfo(ld, "%s_cimv_%s", stmt->into->rel->relname, LANDING_TABLE_SUFFIX); + + Query *query = (Query *) stmt->query; + RangeTblEntry *baseRte = (RangeTblEntry *) linitial(query->rtable); + + cimvCreate->stmt = stmt; + cimvCreate->createOptions = createOptions; + cimvCreate->baseTableName = makeRangeVar(get_namespace_name(get_rel_namespace( + baseRte->relid)), + get_rel_name(baseRte->relid), -1); + cimvCreate->formCimv->basetable = baseRte->relid; + cimvCreate->matTableName = makeRangeVar(CITUS_INTERNAL_SCHEMA, mat->data, -1); + cimvCreate->userViewName = stmt->into->rel; + cimvCreate->refreshViewName = makeRangeVar(CITUS_INTERNAL_SCHEMA, rv->data, -1); + cimvCreate->landingTableName = makeRangeVar(CITUS_INTERNAL_SCHEMA, ld->data, -1); + cimvCreate->targetListEntries = NIL; + cimvCreate->groupTargetListEntries = NIL; + cimvCreate->aggTargetListEntries = NIL; + cimvCreate->citusTable = IsCitusTable(baseRte->relid) ? LookupCitusTableCacheEntry( + baseRte->relid) : NULL; + cimvCreate->partitionColumn = NULL; + cimvCreate->supportsDelete = false; + + cimvCreate->baseTableNameQuoted = makeRangeVar( + (char *) quote_identifier(cimvCreate->baseTableName->schemaname), + (char *) quote_identifier(cimvCreate->baseTableName->relname), -1); + cimvCreate->matTableNameQuoted = makeRangeVar( + (char *) quote_identifier(cimvCreate->matTableName->schemaname), + (char *) quote_identifier(cimvCreate->matTableName->relname), -1); + cimvCreate->userViewNameQuoted = makeRangeVar( + cimvCreate->userViewName->schemaname == NULL ? NULL : (char *) quote_identifier( + cimvCreate->userViewName->schemaname), + (char *) quote_identifier(cimvCreate->userViewName->relname), -1); + cimvCreate->refreshViewNameQuoted = makeRangeVar( + (char *) quote_identifier(cimvCreate->refreshViewName->schemaname), + (char *) quote_identifier(cimvCreate->refreshViewName->relname), -1); + cimvCreate->landingTableNameQuoted = makeRangeVar( + (char *) quote_identifier(cimvCreate->landingTableName->schemaname), + (char *) quote_identifier(cimvCreate->landingTableName->relname), -1); + + return cimvCreate; +} + + +static MatViewCreateOptions * +GetMatViewCreateOptions(const CreateTableAsStmt *stmt) +{ + MatViewCreateOptions *result = palloc(sizeof(MatViewCreateOptions)); + + result->isCimv = false; + result->schedule = NULL; + result->storageOptions = NIL; + + if (stmt == NULL || stmt->into == NULL || stmt->into->options == NIL) + { + return result; + } + + ListCell *lc; + foreach(lc, stmt->into->options) + { + DefElem *def = (DefElem *) lfirst(lc); + + if (def->defnamespace != NULL && pg_strcasecmp(def->defnamespace, + CITUS_NAMESPACE) == 0) + { + if (pg_strcasecmp(def->defname, "cimv") == 0) + { + result->isCimv = defGetBoolean(def); + } + else if (pg_strcasecmp(def->defname, "schedule") == 0) + { + result->schedule = defGetString(def); + } + else if (pg_strcasecmp(def->defname, "insertonlycapture") == 0) + { + result->insertOnlyCapture = defGetBoolean(def); + } + else + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid option \"%s\" in WITH clause", def->defname))); + } + } + else + { + result->storageOptions = lappend(result->storageOptions, def); + } + } + + return result; +} + + +static ObjectAddress +DefineVirtualRelation(RangeVar *relation, List *tlist, Query *viewParse) +{ + CreateStmt *createStmt = makeNode(CreateStmt); + TargetEntry *tle; + + /* + * create a list of ColumnDef nodes based on the names and types of the + * (non-junk) targetlist items from the view's SELECT list. + */ + List *attrList = NIL; + foreach_ptr(tle, tlist) + { + if (!tle->resjunk) + { + ColumnDef *def = makeColumnDef(tle->resname, + exprType((Node *) tle->expr), + exprTypmod((Node *) tle->expr), + exprCollation((Node *) tle->expr)); + + attrList = lappend(attrList, def); + } + } + + /* + * Set the parameters for keys/inheritance etc. All of these are + * uninteresting for views... + */ + createStmt->relation = relation; + createStmt->tableElts = attrList; + createStmt->inhRelations = NIL; + createStmt->constraints = NIL; + createStmt->options = NULL; + createStmt->oncommit = ONCOMMIT_NOOP; + createStmt->tablespacename = NULL; + createStmt->if_not_exists = false; + + /* + * Create the relation (this will error out if there's an existing + * view, so we don't need more code to complain if "replace" is + * false). + */ + ObjectAddress address = DefineRelation(createStmt, RELKIND_VIEW, InvalidOid, NULL, + NULL); + Assert(address.objectId != InvalidOid); + + /* Make the new view relation visible */ + CommandCounterIncrement(); + + /* Store the query for the view */ + StoreViewQuery(address.objectId, viewParse, false); + + CommandCounterIncrement(); + + return address; +} + + +static bool +ValidateCimv(CimvCreate *cimvCreate) +{ + Query *query = (Query *) cimvCreate->stmt->query; + + if (cimvCreate->stmt->into->skipData && !cimvCreate->createOptions->insertOnlyCapture) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("WITH NO DATA requires 'citus.insertonlycapture' set to TRUE"))); + } + + if (query->commandType != CMD_SELECT) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("only SELECT queries allowed"))); + } + + if (query->cteList != NIL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: WITH (common table expressions) not supported"))); + } + + if (query->distinctClause != NIL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: DISTINCT not supported"))); + } + + if (query->groupingSets != NIL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: GROUPING SETS not supported"))); + } + + if (query->hasDistinctOn) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: DISTINCT ON not supported"))); + } + + if (query->hasForUpdate) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: FOR [KEY] UPDATE/SHARE not supported"))); + } + + if (query->hasModifyingCTE) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: INSERT/UPDATE/DELETE in WITH not supported"))); + } + + if (query->hasRecursive) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: WITH RECURSIVE not supported"))); + } + + if (query->hasRowSecurity) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: row-level-security (RLS) policies are not supported"))); + } + + if (query->hasSubLinks) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: subqueries are not supported"))); + } + + if (query->hasTargetSRFs) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: set-returning functions not supported"))); + } + + if (query->hasWindowFuncs) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: window functions (OVER) not supported"))); + } + + if (query->havingQual) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: HAVING not supported"))); + } + + if (query->limitCount != NULL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: LIMIT not supported"))); + } + + if (query->limitOffset != NULL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: OFFSET not supported"))); + } + + if (query->setOperations != NULL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: set-operations (UNION/INTERSECT/EXCEPT) not supported"))); + } + + if (query->sortClause != NIL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: ORDER BY not supported"))); + } + + if (query->groupClause == NIL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: at least one GROUP BY statement is required"))); + } + + if (!query->hasAggs) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: at least one aggregate function is required"))); + } + + if (list_length(query->jointree->fromlist) != 1 || + !IsA(linitial(query->jointree->fromlist), RangeTblRef)) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: materialized view can only be created on single table"))); + } + + RangeTblRef *rtref = linitial_node(RangeTblRef, query->jointree->fromlist); + RangeTblEntry *rte = list_nth(query->rtable, rtref->rtindex - 1); + + if (rte->relkind != RELKIND_RELATION && rte->relkind != RELKIND_PARTITIONED_TABLE) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: materialized view can only be defined on a table"))); + } + + if (rte->tablesample != NULL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "invalid SELECT query: materialized view cannot be defined on a table sample"))); + } + + if (!rte->inh) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("invalid SELECT query: 'SELECT ... FROM ONLY' not supported"))); + } + + cimvCreate->supportsDelete = true; + TargetEntry *tle; + foreach_ptr(tle, query->targetList) + { + if (contain_mutable_functions((Node *) tle)) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("only immutable functions are allowed."))); + } + + if (IsA(tle->expr, Aggref)) + { + Aggref *agg = (Aggref *) tle->expr; + bool supportsDelete; + ValidateAgg(agg, &supportsDelete); + if (!supportsDelete) + { + cimvCreate->supportsDelete = false; + } + cimvCreate->aggTargetListEntries = lappend(cimvCreate->aggTargetListEntries, + tle); + cimvCreate->targetListEntries = lappend(cimvCreate->targetListEntries, tle); + } + else if (!tle->resjunk) + { + if (contain_agg_clause((Node *) tle)) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("only top-level aggregates are allowed."))); + } + + if (cimvCreate->citusTable != NULL && + tle->resorigtbl != InvalidOid && + IsA(tle->expr, Var) && + tle->resorigcol == cimvCreate->citusTable->partitionColumn->varattno) + { + cimvCreate->partitionColumn = tle; + } + + cimvCreate->groupTargetListEntries = lappend( + cimvCreate->groupTargetListEntries, tle); + cimvCreate->targetListEntries = lappend(cimvCreate->targetListEntries, tle); + } + } + + if (cimvCreate->citusTable != NULL && cimvCreate->partitionColumn == NULL) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "SELECT query needs to GROUP BY distribution column of distributed table"))); + } + + return true; +} + + +static void +ValidateAgg(Aggref *agg, bool *supportsDelete) +{ + if (agg->aggorder) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("aggregates with ORDER BY are not supported"))); + } + + if (agg->aggdistinct) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("aggregates with DISTINCT are not supported"))); + } + + if (agg->aggfilter) + { + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("aggregates with FILTER are not supported"))); + } + + HeapTuple aggtuple = SearchSysCache1(AGGFNOID, agg->aggfnoid); + if (!HeapTupleIsValid(aggtuple)) + { + elog(ERROR, "cache lookup failed for aggregate function %u", agg->aggfnoid); + } + + Form_pg_aggregate aggform = (Form_pg_aggregate) GETSTRUCT(aggtuple); + if (aggform->aggkind != 'n') + { + ReleaseSysCache(aggtuple); + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "ordered-set and hypothetical-set aggregates are not supported"))); + } + + if (aggform->aggcombinefn == InvalidOid) + { + ReleaseSysCache(aggtuple); + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("aggregates without COMBINEFUNC are not supported"))); + } + + if (aggform->aggtranstype == INTERNALOID && + (aggform->aggserialfn == InvalidOid || aggform->aggdeserialfn == InvalidOid)) + { + ReleaseSysCache(aggtuple); + ereport(ERROR, + (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "aggregates without INTERNAL stype must specify (de-)serialization functions"))); + } + + ReleaseSysCache(aggtuple); + + *supportsDelete = aggform->aggtransfn == aggform->aggmtransfn && + aggform->aggminvtransfn != InvalidOid; +} + + +static void +AddCountAgg(Query *query, bool isInsert) +{ + Aggref *countAggregate = makeNode(Aggref); + countAggregate->aggfnoid = AggregateFunctionOid("count", ANYOID); + countAggregate->aggtype = get_func_rettype(countAggregate->aggfnoid); + countAggregate->aggtranstype = InvalidOid; + countAggregate->aggargtypes = NIL; + countAggregate->aggsplit = AGGSPLIT_SIMPLE; + countAggregate->aggkind = 'n'; + countAggregate->aggstar = true; + + if (isInsert) + { + TargetEntry *countAggregateTe = makeTargetEntry((Expr *) countAggregate, + query->targetList->length + 1, + "__count__", false); + query->targetList = lappend(query->targetList, countAggregateTe); + } + else + { + OpExpr *negateOp = makeNode(OpExpr); + negateOp->args = list_make1(countAggregate); + negateOp->opno = OpernameGetOprid(list_make2(makeString("pg_catalog"), makeString( + "-")), InvalidOid, INT8OID); + negateOp->opresulttype = INT8OID; + negateOp->opretset = false; + negateOp->opcollid = 0; + negateOp->inputcollid = 0; + set_opfuncid(negateOp); + + TargetEntry *negCountAggregateTe = makeTargetEntry((Expr *) negateOp, + query->targetList->length + 1, + "__count__", false); + query->targetList = lappend(query->targetList, negCountAggregateTe); + } +} + + +static Node * +PartializeAggs(Node *node, void *context) +{ + if (node == NULL) + { + return false; + } + + if (IsA(node, Aggref)) + { + Aggref *originalAggregate = (Aggref *) node; + Oid workerPartialId = PartialAggOid(); + + Const *aggOidParam = makeConst(REGPROCEDUREOID, -1, InvalidOid, sizeof(Oid), + ObjectIdGetDatum(originalAggregate->aggfnoid), + false, true); + + + List *newWorkerAggregateArgs = + list_make1(makeTargetEntry((Expr *) aggOidParam, 1, NULL, false)); + + if (list_length(originalAggregate->args) == 1) + { + /* + * Single argument case, append 'arg' to partial_agg(agg, arg). + * We don't wrap single argument in a row expression because + * it has performance implications to unwrap arguments on each + * SFUNC invocation. + */ + TargetEntry *newArg = + copyObject((TargetEntry *) linitial(originalAggregate->args)); + newArg->resno++; + newWorkerAggregateArgs = lappend(newWorkerAggregateArgs, newArg); + } + else + { + /* + * Aggregation on workers assumes a single aggregation parameter. + * To still be able to handle multiple parameters, we combine + * parameters into a single row expression, i.e., append 'ROW(...args)' + * to partial_agg(agg, ROW(...args)). + */ + RowExpr *rowExpr = makeNode(RowExpr); + rowExpr->row_typeid = RECORDOID; + rowExpr->row_format = COERCE_EXPLICIT_CALL; + rowExpr->location = -1; + rowExpr->colnames = NIL; + + TargetEntry *arg = NULL; + foreach_ptr(arg, originalAggregate->args) + { + rowExpr->args = lappend(rowExpr->args, copyObject(arg->expr)); + } + + newWorkerAggregateArgs = + lappend(newWorkerAggregateArgs, + makeTargetEntry((Expr *) rowExpr, 2, NULL, false)); + } + + Const *inverseParam = makeConst(INT4OID, -1, InvalidOid, sizeof(INT4OID), + Int32GetDatum(*((int *) context)), + false, true); + + newWorkerAggregateArgs = + lappend(newWorkerAggregateArgs, + makeTargetEntry((Expr *) inverseParam, 3, NULL, false)); + + /* orker_partial_agg(agg, arg) or partial_agg(agg, ROW(...args)) */ + Aggref *newWorkerAggregate = copyObject(originalAggregate); + newWorkerAggregate->aggfnoid = workerPartialId; + newWorkerAggregate->aggtype = BYTEAOID; + newWorkerAggregate->args = newWorkerAggregateArgs; + newWorkerAggregate->aggkind = AGGKIND_NORMAL; + newWorkerAggregate->aggtranstype = INTERNALOID; + newWorkerAggregate->aggargtypes = lappend_oid(lcons_oid(OIDOID, + newWorkerAggregate-> + aggargtypes), INT4OID); + newWorkerAggregate->aggsplit = AGGSPLIT_SIMPLE; + + return (Node *) newWorkerAggregate; + } + return expression_tree_mutator(node, PartializeAggs, context); +} + + +/* + * CitusFunctionOidWithSignature looks up a function with given input types. + * Looks in pg_catalog schema, as this function's sole purpose is + * support aggregate lookup. + */ +static Oid +CitusFunctionOidWithSignature(char *functionName, int numargs, Oid *argtypes) +{ + List *aggregateName = list_make2(makeString("pg_catalog"), makeString(functionName)); + FuncCandidateList clist = FuncnameGetCandidates(aggregateName, numargs, NIL, false, + false, true); + + for (; clist; clist = clist->next) + { + if (memcmp(clist->args, argtypes, numargs * sizeof(Oid)) == 0) + { + return clist->oid; + } + } + + ereport(ERROR, (errmsg("no matching oid for function: %s", functionName))); + return InvalidOid; +} + + +/* + * PartialAggOid looks up oid of pg_catalog.partial_agg + */ +static Oid +PartialAggOid() +{ + Oid argtypes[] = { + OIDOID, + ANYELEMENTOID, + INT4OID + }; + + return CitusFunctionOidWithSignature(PARTIAL_AGG_FNNAME, 3, argtypes); +} + + +static void +AppendQuotedLiteral(StringInfo buf, const char *val) +{ + /* + * We form the string literal according to the prevailing setting of + * standard_conforming_strings; we never use E''. User is responsible for + * making sure result is used correctly. + */ + appendStringInfoChar(buf, '\''); + for (const char *valptr = val; *valptr; valptr++) + { + char ch = *valptr; + + if (SQL_STR_DOUBLE(ch, !standard_conforming_strings)) + { + appendStringInfoChar(buf, ch); + } + appendStringInfoChar(buf, ch); + } + appendStringInfoChar(buf, '\''); +} + + +static void +AppendStringInfoFunction(StringInfo buf, Oid fnoid) +{ + Oid typoutput; + bool typIsVarlena; + + getTypeOutputInfo(REGPROCEDUREOID, + &typoutput, &typIsVarlena); + + char *extval = OidOutputFunctionCall(typoutput, ObjectIdGetDatum(fnoid)); + + AppendQuotedLiteral(buf, extval); + + pfree(extval); + + appendStringInfo(buf, "::%s", + format_type_with_typemod(REGPROCEDUREOID, + -1)); +} + + +/* AggregateFunctionOid performs a reverse lookup on aggregate function name, + * and returns the corresponding aggregate function oid for the given function + * name and input type. + */ +static Oid +AggregateFunctionOid(const char *functionName, Oid inputType) +{ + Oid functionOid = InvalidOid; + ScanKeyData scanKey[1]; + int scanKeyCount = 1; + + Relation procRelation = table_open(ProcedureRelationId, AccessShareLock); + + ScanKeyInit(&scanKey[0], Anum_pg_proc_proname, + BTEqualStrategyNumber, F_NAMEEQ, CStringGetDatum(functionName)); + + SysScanDesc scanDescriptor = systable_beginscan(procRelation, + ProcedureNameArgsNspIndexId, true, + NULL, scanKeyCount, scanKey); + + /* loop until we find the right function */ + HeapTuple heapTuple = systable_getnext(scanDescriptor); + while (HeapTupleIsValid(heapTuple)) + { + Form_pg_proc procForm = (Form_pg_proc) GETSTRUCT(heapTuple); + int argumentCount = procForm->pronargs; + + if (argumentCount == 1) + { + /* check if input type and found value type match */ + if (procForm->proargtypes.values[0] == inputType || + procForm->proargtypes.values[0] == ANYELEMENTOID) + { +#if PG_VERSION_NUM < PG_VERSION_12 + functionOid = HeapTupleGetOid(heapTuple); +#else + functionOid = procForm->oid; +#endif + break; + } + } + Assert(argumentCount <= 1); + + heapTuple = systable_getnext(scanDescriptor); + } + + if (functionOid == InvalidOid) + { + ereport(ERROR, (errmsg("no matching oid for function: %s", functionName))); + } + + systable_endscan(scanDescriptor); + table_close(procRelation, AccessShareLock); + + return functionOid; +} diff --git a/src/backend/distributed/cimv/drop.c b/src/backend/distributed/cimv/drop.c new file mode 100644 index 000000000..6a0dd9490 --- /dev/null +++ b/src/backend/distributed/cimv/drop.c @@ -0,0 +1,245 @@ +#include "postgres.h" + +#include "catalog/dependency.h" +#include "catalog/objectaddress.h" +#include "catalog/pg_class.h" +#include "distributed/listutils.h" +#include "distributed/metadata_cache.h" +#include "distributed/pg_cimv.h" +#include "executor/spi.h" +#include "nodes/parsenodes.h" +#include "utils/builtins.h" +#include "utils/elog.h" +#include "utils/lsyscache.h" +#include "utils/syscache.h" + +#include "distributed/cimv.h" + +static void DropCimv(Form_pg_cimv formCimv, DropBehavior behavior); +static void DropDmlTriggers(Form_pg_cimv cimv); +static void DropCronJob(Form_pg_cimv cimv); + +extern void +ProcessDropMaterializedViewStmt(DropStmt *stmt) +{ + if (stmt->removeType != OBJECT_MATVIEW) + { + return; + } + + List *viewNameList = NULL; + List *cimvObjects = NIL; + List *formCimvs = NIL; + Form_pg_cimv formCimv; + bool hasCimv = false; + bool hasNonCimv = false; + foreach_ptr(viewNameList, stmt->objects) + { + RangeVar *viewRangeVar = makeRangeVarFromNameList(viewNameList); + + Oid relationId = RangeVarGetRelid(viewRangeVar, NoLock, true); + + if (relationId == InvalidOid) + { + hasNonCimv = true; + continue; + } + + formCimv = LookupCimvFromCatalog(relationId, true); + + if (formCimv == NULL) + { + hasNonCimv = true; + continue; + } + + hasCimv = true; + + cimvObjects = lappend(cimvObjects, list_make2(makeString(get_namespace_name( + get_rel_namespace( + formCimv-> + mattable))), + makeString(get_rel_name( + formCimv + -> + mattable)))); + formCimvs = lappend(formCimvs, formCimv); + } + + if (hasCimv && hasNonCimv) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg( + "cannot drop regular and Citus materialized views with single command"))); + } + + if (hasCimv) + { + foreach_ptr(formCimv, formCimvs) + { + DropCimv(formCimv, stmt->behavior); + } + + stmt->removeType = OBJECT_TABLE; + stmt->objects = cimvObjects; + } +} + + +extern void +ProcessDropViewStmt(DropStmt *stmt) +{ + if (stmt->removeType != OBJECT_VIEW) + { + return; + } + + List *viewNameList = NULL; + foreach_ptr(viewNameList, stmt->objects) + { + RangeVar *viewRangeVar = makeRangeVarFromNameList(viewNameList); + + Oid relationId = RangeVarGetRelid(viewRangeVar, NoLock, true); + + if (relationId == InvalidOid) + { + continue; + } + + Form_pg_cimv formCimv = LookupCimvFromCatalog(relationId, true); + + if (formCimv != NULL) + { + ereport(ERROR, (errcode(ERRCODE_FEATURE_NOT_SUPPORTED), + errmsg("DROP VIEW not supported for %s", + viewRangeVar->relname), + errhint("use DROP MATERIALIZED VIEW"))); + } + } +} + + +static void +DropCimv(Form_pg_cimv formCimv, DropBehavior behavior) +{ + ObjectAddress matTableAddress; + matTableAddress.classId = RelationRelationId; + matTableAddress.objectId = formCimv->mattable; + matTableAddress.objectSubId = 0; + + ObjectAddress userViewAddress; + userViewAddress.classId = RelationRelationId; + userViewAddress.objectId = formCimv->userview; + userViewAddress.objectSubId = 0; + + ObjectAddress refreshViewAddress; + refreshViewAddress.classId = RelationRelationId; + refreshViewAddress.objectId = formCimv->refreshview; + refreshViewAddress.objectSubId = 0; + + ObjectAddress landingTableAddress; + landingTableAddress.classId = RelationRelationId; + landingTableAddress.objectId = formCimv->landingtable; + landingTableAddress.objectSubId = 0; + + + if (SPI_connect() != SPI_OK_CONNECT) + { + elog(ERROR, "SPI_connect failed"); + } + + DropCronJob(formCimv); + DropDmlTriggers(formCimv); + + /* Lock */ + if (OidIsValid(matTableAddress.objectId)) + { + AcquireDeletionLock(&matTableAddress, 0); + } + + if (OidIsValid(userViewAddress.objectId)) + { + AcquireDeletionLock(&userViewAddress, 0); + } + + if (OidIsValid(refreshViewAddress.objectId)) + { + AcquireDeletionLock(&refreshViewAddress, 0); + } + + if (OidIsValid(landingTableAddress.objectId)) + { + AcquireDeletionLock(&landingTableAddress, 0); + } + + /* Drop views */ + if (OidIsValid(userViewAddress.objectId)) + { + performDeletion(&userViewAddress, behavior, 0); + } + + if (OidIsValid(refreshViewAddress.objectId)) + { + performDeletion(&refreshViewAddress, behavior, 0); + } + + /* Drop landing table */ + if (OidIsValid(landingTableAddress.objectId)) + { + performDeletion(&landingTableAddress, behavior, 0); + } + + DeletePgCimvRow(userViewAddress.objectId); + + /* Close SPI context. */ + if (SPI_finish() != SPI_OK_FINISH) + { + elog(ERROR, "SPI_finish failed"); + } +} + + +static void +DropDmlTriggers(Form_pg_cimv cimv) +{ + StringInfoData querybuf; + initStringInfo(&querybuf); + + appendStringInfo(&querybuf, + "DROP FUNCTION %s.%s CASCADE;", + quote_identifier(NameStr(cimv->triggerfnnamespace)), + quote_identifier(NameStr(cimv->triggerfnname))); + + if (SPI_execute(querybuf.data, false, 0) != SPI_OK_UTILITY) + { + elog(ERROR, "SPI_exec failed: %s", querybuf.data); + } + + pfree(querybuf.data); +} + + +static void +DropCronJob(Form_pg_cimv cimv) +{ + if (cimv->jobid == 0) + { + return; + } + + StringInfoData querybuf; + initStringInfo(&querybuf); + + appendStringInfo(&querybuf, + "SELECT * FROM cron.unschedule(" INT64_FORMAT ");", + cimv->jobid); + + int spiResult = SPI_execute(querybuf.data, false, 0); + + if (spiResult != SPI_OK_SELECT) + { + elog(ERROR, "SPI_exec failed: %s", querybuf.data); + } + + pfree(querybuf.data); +} diff --git a/src/backend/distributed/cimv/refresh.c b/src/backend/distributed/cimv/refresh.c new file mode 100644 index 000000000..0d4eae3fd --- /dev/null +++ b/src/backend/distributed/cimv/refresh.c @@ -0,0 +1,193 @@ +#include "postgres.h" + +#include "catalog/dependency.h" +#include "catalog/objectaddress.h" +#include "catalog/pg_class.h" +#include "distributed/listutils.h" +#include "distributed/metadata_cache.h" +#include "distributed/pg_cimv.h" +#include "executor/spi.h" +#include "nodes/parsenodes.h" +#include "utils/builtins.h" +#include "utils/lsyscache.h" +#include "utils/snapmgr.h" + +#include "distributed/cimv.h" + +static void SpiExecuteSnapshot(StringInfo query, Snapshot snapshot, int expectedResult); +static void SpiExecute(StringInfo query, int expectedResult); + +bool +ProcessRefreshMaterializedViewStmt(RefreshMatViewStmt *stmt) +{ + Oid relationId = RangeVarGetRelid(stmt->relation, NoLock, true); + + if (relationId == InvalidOid) + { + return false; + } + + Form_pg_cimv formCimv = LookupCimvFromCatalog(relationId, true); + + if (formCimv == NULL) + { + return false; + } + + RefreshCimv(formCimv, stmt->skipData, false); + return true; +} + + +void +RefreshCimv(Form_pg_cimv formCimv, bool skipData, bool isCreate) +{ + StringInfoData querybuf; + initStringInfo(&querybuf); + + if (SPI_connect_ext(SPI_OPT_NONATOMIC) != SPI_OK_CONNECT) + { + elog(ERROR, "SPI_connect failed"); + } + + const char *matTableSchemaName = get_namespace_name(get_rel_namespace( + formCimv->mattable)); + const char *matTableName = get_rel_name(formCimv->mattable); + matTableSchemaName = quote_identifier(matTableSchemaName); + matTableName = quote_identifier(matTableName); + + const char *landingTableSchemaName = NULL; + const char *landingTableName = NULL; + if (formCimv->landingtable) + { + landingTableSchemaName = get_namespace_name(get_rel_namespace( + formCimv->landingtable)); + landingTableName = get_rel_name(formCimv->landingtable); + landingTableSchemaName = quote_identifier(landingTableSchemaName); + landingTableName = quote_identifier(landingTableName); + } + + if (skipData) + { + if (formCimv->landingtable) + { + appendStringInfo(&querybuf, + "TRUNCATE TABLE %s.%s", + landingTableSchemaName, + landingTableName); + + SpiExecute(&querybuf, SPI_OK_UTILITY); + resetStringInfo(&querybuf); + } + appendStringInfo(&querybuf, + "TRUNCATE TABLE %s.%s", + matTableSchemaName, + matTableName); + + SpiExecute(&querybuf, SPI_OK_UTILITY); + resetStringInfo(&querybuf); + } + else + { + const char *refreshViewSchemaName = get_namespace_name(get_rel_namespace( + formCimv->refreshview)); + const char *refreshViewName = get_rel_name(formCimv->refreshview); + refreshViewSchemaName = quote_identifier(refreshViewSchemaName); + refreshViewName = quote_identifier(refreshViewName); + + if (isCreate) + { + /* better: SPI_commit_and_chain(); */ + SPI_commit(); + SPI_start_transaction(); + + /* TODO: cleanup if this fails */ + appendStringInfo(&querybuf, + "INSERT INTO %s.%s " + "SELECT * FROM %s.%s", + matTableSchemaName, + matTableName, + refreshViewSchemaName, + refreshViewName); + if (SPI_execute(querybuf.data, false, 0) != SPI_OK_INSERT) + { + elog(ERROR, "SPI_exec failed: %s", querybuf.data); + } + } + else + { + Snapshot snapshot = GetLatestSnapshot(); + + /* TODO: DELETE only if !isCreate */ + appendStringInfo(&querybuf, + "DELETE FROM %s.%s", + matTableSchemaName, + matTableName); + SpiExecuteSnapshot(&querybuf, snapshot, SPI_OK_DELETE); + resetStringInfo(&querybuf); + + appendStringInfo(&querybuf, + "INSERT INTO %s.%s " + "SELECT * FROM %s.%s", + matTableSchemaName, + matTableName, + refreshViewSchemaName, + refreshViewName); + SpiExecuteSnapshot(&querybuf, snapshot, SPI_OK_INSERT); + resetStringInfo(&querybuf); + + if (formCimv->landingtable != InvalidOid) + { + /* TODO: DELETE only if !isCreate */ + appendStringInfo(&querybuf, + "DELETE FROM %s.%s", + landingTableSchemaName, + landingTableName); + SpiExecuteSnapshot(&querybuf, snapshot, SPI_OK_DELETE); + resetStringInfo(&querybuf); + } + } + } + + /* Close SPI context. */ + if (SPI_finish() != SPI_OK_FINISH) + { + elog(ERROR, "SPI_finish failed"); + } +} + + +static void +SpiExecuteSnapshot(StringInfo query, Snapshot snapshot, int expectedResult) +{ + SPIPlanPtr qplan = SPI_prepare(query->data, 0, NULL); + + if (qplan == NULL) + { + elog(ERROR, "SPI_prepare returned %s for %s", + SPI_result_code_string(SPI_result), query->data); + } + + int spi_result = SPI_execute_snapshot(qplan, + NULL, NULL, + snapshot, + InvalidSnapshot, + false, false, 0); + + if (spi_result != expectedResult) + { + elog(ERROR, "SPI_exec failed: %s", query->data); + } +} + + +static void +SpiExecute(StringInfo query, int expectedResult) +{ + int spi_result = SPI_execute(query->data, false, 0); + + if (spi_result != expectedResult) + { + elog(ERROR, "SPI_exec failed: %s", query->data); + } +} diff --git a/src/backend/distributed/commands/utility_hook.c b/src/backend/distributed/commands/utility_hook.c index 17253854d..fc762c707 100644 --- a/src/backend/distributed/commands/utility_hook.c +++ b/src/backend/distributed/commands/utility_hook.c @@ -59,6 +59,7 @@ #include "distributed/transmit.h" #include "distributed/version_compat.h" #include "distributed/worker_transaction.h" +#include "distributed/cimv.h" #include "lib/stringinfo.h" #include "nodes/parsenodes.h" #include "nodes/pg_list.h" @@ -457,6 +458,35 @@ multi_ProcessUtility(PlannedStmt *pstmt, } } + bool continueProcessing = true; + if (IsA(parsetree, CreateTableAsStmt)) + { + continueProcessing = !ProcessCreateMaterializedViewStmt((const + CreateTableAsStmt *) + parsetree, queryString, + pstmt); + } + + if (IsA(parsetree, RefreshMatViewStmt)) + { + continueProcessing = !ProcessRefreshMaterializedViewStmt( + (RefreshMatViewStmt *) parsetree); + } + + if (IsA(parsetree, DropStmt)) + { + DropStmt *dropStatement = (DropStmt *) parsetree; + + if (dropStatement->removeType == OBJECT_MATVIEW) + { + ProcessDropMaterializedViewStmt(dropStatement); + } + else if (dropStatement->removeType == OBJECT_VIEW) + { + ProcessDropViewStmt(dropStatement); + } + } + if (IsDropCitusExtensionStmt(parsetree)) { StopMaintenanceDaemon(MyDatabaseId); @@ -483,8 +513,11 @@ multi_ProcessUtility(PlannedStmt *pstmt, citusCanBeUpdatedToAvailableVersion = !InstalledAndAvailableVersionsSame(); } - standard_ProcessUtility(pstmt, queryString, context, - params, queryEnv, dest, completionTag); + if (continueProcessing) + { + standard_ProcessUtility(pstmt, queryString, context, + params, queryEnv, dest, completionTag); + } /* * if we are running ALTER EXTENSION citus UPDATE (to "") command, we may need diff --git a/src/backend/distributed/metadata/metadata_cache.c b/src/backend/distributed/metadata/metadata_cache.c index bc5c809ba..fa2fc22b7 100644 --- a/src/backend/distributed/metadata/metadata_cache.c +++ b/src/backend/distributed/metadata/metadata_cache.c @@ -33,6 +33,7 @@ #include "commands/trigger.h" #include "distributed/colocation_utils.h" #include "distributed/connection_management.h" +#include "distributed/cimv.h" #include "distributed/citus_ruleutils.h" #include "distributed/function_utils.h" #include "distributed/foreign_key_relationship.h" @@ -140,6 +141,8 @@ typedef struct MetadataCacheData Oid distColocationRelationId; Oid distColocationConfigurationIndexId; Oid distPartitionRelationId; + Oid pgCimvId; + Oid pgCimvIndexId; Oid distPartitionLogicalRelidIndexId; Oid distPartitionColocationidIndexId; Oid distShardLogicalRelidIndexId; @@ -2118,6 +2121,28 @@ DistPartitionRelationId(void) } +/* return oid of pg_cimv relation */ +Oid +PgCimvId(void) +{ + CachedRelationLookup("pg_cimv", + &MetadataCache.pgCimvId); + + return MetadataCache.pgCimvId; +} + + +/* return oid of pg_cimv_pkey index */ +Oid +PgCimvIndexId(void) +{ + CachedRelationLookup("pg_cimv_pkey", + &MetadataCache.pgCimvIndexId); + + return MetadataCache.pgCimvIndexId; +} + + /* return oid of pg_dist_partition_logical_relid_index index */ Oid DistPartitionLogicalRelidIndexId(void) diff --git a/src/backend/distributed/metadata/metadata_utility.c b/src/backend/distributed/metadata/metadata_utility.c index 10a4ff5b6..f671aeab6 100644 --- a/src/backend/distributed/metadata/metadata_utility.c +++ b/src/backend/distributed/metadata/metadata_utility.c @@ -46,6 +46,7 @@ #include "distributed/pg_dist_partition.h" #include "distributed/pg_dist_shard.h" #include "distributed/pg_dist_placement.h" +#include "distributed/pg_cimv.h" #include "distributed/reference_table_utils.h" #include "distributed/relay_utility.h" #include "distributed/resource_lock.h" @@ -1031,6 +1032,110 @@ InsertIntoPgDistPartition(Oid relationId, char distributionMethod, } +/* + * InsertIntoPgCimv inserts a new tuple into pg_cimv. + */ +void +InsertIntoPgCimv(Form_pg_cimv cimv) +{ + Datum newValues[Natts_pg_cimv]; + bool newNulls[Natts_pg_cimv]; + + /* open system catalog and insert new tuple */ + Relation pgCimv = table_open(PgCimvId(), RowExclusiveLock); + + /* form new tuple for pg_cimv */ + memset(newValues, 0, sizeof(newValues)); + memset(newNulls, false, sizeof(newNulls)); + + newValues[Anum_pg_cimv_userview - 1] = ObjectIdGetDatum(cimv->userview); + newValues[Anum_pg_cimv_basetable - 1] = ObjectIdGetDatum(cimv->basetable); + newValues[Anum_pg_cimv_mattable - 1] = ObjectIdGetDatum(cimv->mattable); + newValues[Anum_pg_cimv_refreshview - 1] = ObjectIdGetDatum(cimv->refreshview); + newValues[Anum_pg_cimv_triggernamespace - 1] = NameGetDatum( + &cimv->triggerfnnamespace); + newValues[Anum_pg_cimv_triggername - 1] = NameGetDatum(&cimv->triggerfnname); + newValues[Anum_pg_cimv_landingtable - 1] = ObjectIdGetDatum(cimv->landingtable); + newValues[Anum_pg_cimv_jobid - 1] = Int64GetDatum(cimv->jobid); + + HeapTuple newTuple = heap_form_tuple(RelationGetDescr(pgCimv), newValues, + newNulls); + + /* finally insert tuple, build index entries & register cache invalidation */ + CatalogTupleInsert(pgCimv, newTuple); + + + CommandCounterIncrement(); + table_close(pgCimv, NoLock); +} + + +void +DeletePgCimvRow(Oid userViewId) +{ + ScanKeyData scanKey[1]; + int scanKeyCount = 1; + + Relation pgCimv = table_open(PgCimvId(), RowExclusiveLock); + + ScanKeyInit(&scanKey[0], Anum_pg_cimv_userview, + BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(userViewId)); + + SysScanDesc scanDescriptor = systable_beginscan(pgCimv, InvalidOid, false, + NULL, + scanKeyCount, scanKey); + + HeapTuple heapTuple = systable_getnext(scanDescriptor); + if (!HeapTupleIsValid(heapTuple)) + { + ereport(ERROR, (errmsg("could not find valid entry for cimv %d", + userViewId))); + } + + simple_heap_delete(pgCimv, &heapTuple->t_self); + + systable_endscan(scanDescriptor); + + /* increment the counter so that next command can see the row */ + CommandCounterIncrement(); + + table_close(pgCimv, NoLock); +} + + +Form_pg_cimv +LookupCimvFromCatalog(Oid userViewId, bool missingOk) +{ + ScanKeyData scanKey[1]; + int scanKeyCount = 1; + Form_pg_cimv cimvForm = NULL; + Relation pgCimv = table_open(PgCimvId(), AccessShareLock); + + ScanKeyInit(&scanKey[0], Anum_pg_cimv_userview, + BTEqualStrategyNumber, F_OIDEQ, ObjectIdGetDatum(userViewId)); + + SysScanDesc scanDescriptor = systable_beginscan(pgCimv, + PgCimvIndexId(), true, + NULL, scanKeyCount, scanKey); + + HeapTuple heapTuple = systable_getnext(scanDescriptor); + if (!HeapTupleIsValid(heapTuple) && !missingOk) + { + ereport(ERROR, (errmsg("could not find valid entry for view %u", userViewId))); + } + + if (HeapTupleIsValid(heapTuple)) + { + cimvForm = (Form_pg_cimv) GETSTRUCT(heapTuple); + } + + systable_endscan(scanDescriptor); + table_close(pgCimv, NoLock); + + return cimvForm; +} + + /* * RecordDistributedRelationDependencies creates the dependency entries * necessary for a distributed relation in addition to the preexisting ones diff --git a/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql b/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql index 2e8a7351e..82c381c98 100644 --- a/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql +++ b/src/backend/distributed/sql/citus--9.4-1--9.5-1.sql @@ -20,4 +20,82 @@ DROP TRIGGER dist_poolinfo_task_tracker_cache_invalidate ON pg_catalog.pg_dist_p DROP FUNCTION task_tracker_conninfo_cache_invalidate(); DROP FUNCTION master_drop_sequences(text[]); +CREATE FUNCTION pg_catalog.partial_agg_sfunc(internal, oid, anyelement, int) +RETURNS internal +AS 'MODULE_PATHNAME' +LANGUAGE C PARALLEL SAFE; + +CREATE FUNCTION pg_catalog.partial_agg_ffunc(internal) +RETURNS bytea +AS 'MODULE_PATHNAME' +LANGUAGE C PARALLEL SAFE; + +CREATE AGGREGATE pg_catalog.partial_agg(oid, anyelement, int) ( + STYPE = internal, + SFUNC = pg_catalog.partial_agg_sfunc, + FINALFUNC = pg_catalog.partial_agg_ffunc +); + +CREATE FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea, anyelement) +RETURNS internal +AS 'MODULE_PATHNAME' +LANGUAGE C PARALLEL SAFE; + +CREATE FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea) +RETURNS internal +AS 'MODULE_PATHNAME' +LANGUAGE C PARALLEL SAFE; + +CREATE FUNCTION pg_catalog.combine_agg_ffunc(internal, oid, bytea, anyelement) +RETURNS anyelement +AS 'MODULE_PATHNAME' +LANGUAGE C PARALLEL SAFE; + +CREATE AGGREGATE pg_catalog.combine_agg(oid, bytea) ( + STYPE = internal, + SFUNC = pg_catalog.combine_agg_sfunc, + FINALFUNC = pg_catalog.partial_agg_ffunc + +); + +CREATE AGGREGATE pg_catalog.finalize_agg(oid, bytea, anyelement) ( + STYPE = internal, + SFUNC = pg_catalog.combine_agg_sfunc, + FINALFUNC = pg_catalog.combine_agg_ffunc, + FINALFUNC_EXTRA +); + +REVOKE ALL ON FUNCTION pg_catalog.partial_agg_sfunc FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.partial_agg_ffunc FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.partial_agg FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea, anyelement) FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea) FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.combine_agg_ffunc FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.combine_agg FROM PUBLIC; +REVOKE ALL ON FUNCTION pg_catalog.finalize_agg FROM PUBLIC; + +GRANT EXECUTE ON FUNCTION pg_catalog.partial_agg_sfunc TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.partial_agg_ffunc TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.partial_agg TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea, anyelement) TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea) TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.combine_agg_ffunc TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.combine_agg TO PUBLIC; +GRANT EXECUTE ON FUNCTION pg_catalog.finalize_agg TO PUBLIC; + +-- add pg_cimv TODO: finalize name of this table +CREATE TABLE citus.pg_cimv( + userview regclass NOT NULL PRIMARY KEY, + basetable regclass NOT NULL, + mattable regclass NOT NULL, + refreshview regclass NOT NULL, + deltatriggerfnnamespace name NOT NULL, + deltatriggerfnname name NOT NULL, + landingtable regclass NOT NULL, + jobid bigint NOT NULL +); + +ALTER TABLE citus.pg_cimv SET SCHEMA pg_catalog; +GRANT SELECT ON pg_catalog.pg_cimv TO public; + RESET search_path; diff --git a/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql b/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql index bb3d45be9..9fc89a863 100644 --- a/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql +++ b/src/backend/distributed/sql/downgrades/citus--9.5-1--9.4-1.sql @@ -92,3 +92,14 @@ COMMENT ON FUNCTION master_drop_sequences(text[]) RESET search_path; DROP FUNCTION pg_catalog.undistribute_table(table_name regclass); + +DROP AGGREGATE pg_catalog.finalize_agg(oid, bytea, anyelement); +DROP AGGREGATE pg_catalog.combine_agg(oid, bytea); +DROP FUNCTION pg_catalog.combine_agg_ffunc(internal, oid, bytea, anyelement); +DROP FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea); +DROP FUNCTION pg_catalog.combine_agg_sfunc(internal, oid, bytea, anyelement); +DROP AGGREGATE pg_catalog.partial_agg(oid, anyelement, int); +DROP FUNCTION pg_catalog.partial_agg_ffunc(internal); +DROP FUNCTION pg_catalog.partial_agg_sfunc(internal, oid, anyelement, int); + +DROP TABLE pg_catalog.pg_cimv; diff --git a/src/backend/distributed/utils/aggregate_utils.c b/src/backend/distributed/utils/aggregate_utils.c index 91f6fc523..352e5e74f 100644 --- a/src/backend/distributed/utils/aggregate_utils.c +++ b/src/backend/distributed/utils/aggregate_utils.c @@ -22,9 +22,11 @@ #include "catalog/pg_type.h" #include "distributed/version_compat.h" #include "nodes/nodeFuncs.h" +#include "parser/parse_agg.h" #include "utils/acl.h" #include "utils/builtins.h" #include "utils/datum.h" +#include "utils/expandeddatum.h" #include "utils/lsyscache.h" #include "utils/syscache.h" #include "utils/typcache.h" @@ -37,6 +39,11 @@ PG_FUNCTION_INFO_V1(worker_partial_agg_ffunc); PG_FUNCTION_INFO_V1(coord_combine_agg_sfunc); PG_FUNCTION_INFO_V1(coord_combine_agg_ffunc); +PG_FUNCTION_INFO_V1(partial_agg_sfunc); +PG_FUNCTION_INFO_V1(partial_agg_ffunc); +PG_FUNCTION_INFO_V1(combine_agg_sfunc); +PG_FUNCTION_INFO_V1(combine_agg_ffunc); + /* * Holds information describing the structure of aggregation arguments * and helps to efficiently handle both a single argument and multiple @@ -73,6 +80,11 @@ typedef struct StypeBox AggregationArgumentContext *aggregationArgumentContext; } StypeBox; +/* TODO: replace PG_FUNCTION_ARGS with explicit parameters */ +static Datum partial_agg_sfunc_internal(PG_FUNCTION_ARGS); +static Datum partial_agg_ffunc_internal(PG_FUNCTION_ARGS, bool serializeToCString); +static Datum combine_agg_sfunc_internal(PG_FUNCTION_ARGS, bool deserializeFromCString); +static Datum combine_agg_ffunc_internal(PG_FUNCTION_ARGS); static HeapTuple GetAggregateForm(Oid oid, Form_pg_aggregate *form); static HeapTuple GetProcForm(Oid oid, Form_pg_proc *form); static HeapTuple GetTypeForm(Oid oid, Form_pg_type *form); @@ -482,6 +494,62 @@ HandleStrictUninit(StypeBox *box, FunctionCallInfo fcinfo, Datum value) } +Datum +worker_partial_agg_sfunc(PG_FUNCTION_ARGS) +{ + return partial_agg_sfunc_internal(fcinfo); +} + + +Datum +worker_partial_agg_ffunc(PG_FUNCTION_ARGS) +{ + return partial_agg_ffunc_internal(fcinfo, true); +} + + +Datum +coord_combine_agg_sfunc(PG_FUNCTION_ARGS) +{ + return combine_agg_sfunc_internal(fcinfo, true); +} + + +Datum +coord_combine_agg_ffunc(PG_FUNCTION_ARGS) +{ + return combine_agg_ffunc_internal(fcinfo); +} + + +Datum +partial_agg_sfunc(PG_FUNCTION_ARGS) +{ + return partial_agg_sfunc_internal(fcinfo); +} + + +Datum +partial_agg_ffunc(PG_FUNCTION_ARGS) +{ + return partial_agg_ffunc_internal(fcinfo, false); +} + + +Datum +combine_agg_sfunc(PG_FUNCTION_ARGS) +{ + return combine_agg_sfunc_internal(fcinfo, false); +} + + +Datum +combine_agg_ffunc(PG_FUNCTION_ARGS) +{ + return combine_agg_ffunc_internal(fcinfo); +} + + /* * worker_partial_agg_sfunc advances transition state, * essentially implementing the following pseudocode: @@ -492,7 +560,7 @@ HandleStrictUninit(StypeBox *box, FunctionCallInfo fcinfo, Datum value) * return box */ Datum -worker_partial_agg_sfunc(PG_FUNCTION_ARGS) +partial_agg_sfunc_internal(PG_FUNCTION_ARGS) { StypeBox *box = NULL; Form_pg_aggregate aggform; @@ -500,6 +568,7 @@ worker_partial_agg_sfunc(PG_FUNCTION_ARGS) FmgrInfo info; int argumentIndex = 0; bool initialCall = PG_ARGISNULL(0); + int mode = PG_NARGS() < 4 ? 0 : PG_GETARG_INT32(3); if (initialCall) { @@ -525,7 +594,25 @@ worker_partial_agg_sfunc(PG_FUNCTION_ARGS) } HeapTuple aggtuple = GetAggregateForm(box->agg, &aggform); - Oid aggsfunc = aggform->aggtransfn; + + if (mode == 1 && + (aggform->aggminvtransfn == 0 || + aggform->aggtransfn != aggform->aggmtransfn)) + { + ereport(ERROR, (errmsg("applying inverse state transition not supported " + "for this aggregate"))); + } + + Oid aggsfunc = InvalidOid; + if (mode == 1) + { + aggsfunc = aggform->aggminvtransfn; + } + else + { + aggsfunc = aggform->aggtransfn; + } + if (initialCall) { @@ -604,11 +691,11 @@ worker_partial_agg_sfunc(PG_FUNCTION_ARGS) * worker_partial_agg_ffunc serializes transition state, * essentially implementing the following pseudocode: * - * (box) -> text + * (box) -> text/byte * return box.agg.stype.output(box.value) */ Datum -worker_partial_agg_ffunc(PG_FUNCTION_ARGS) +partial_agg_ffunc_internal(PG_FUNCTION_ARGS, bool serializeToCString) { LOCAL_FCINFO(innerFcinfo, 1); FmgrInfo info; @@ -635,27 +722,66 @@ worker_partial_agg_ffunc(PG_FUNCTION_ARGS) "worker_partial_agg_ffunc expects an aggregate with COMBINEFUNC"))); } - if (aggform->aggtranstype == INTERNALOID) + if (serializeToCString && aggform->aggtranstype == INTERNALOID) { ereport(ERROR, (errmsg( "worker_partial_agg_ffunc does not support aggregates with INTERNAL transition state"))); } - Oid transtype = aggform->aggtranstype; + ReleaseSysCache(aggtuple); - getTypeOutputInfo(transtype, &typoutput, &typIsVarlena); + Datum result = (Datum) 0; + bool isNull = false; - fmgr_info(typoutput, &info); + /* TODO: move function call info into box to avoid initializing it repeatedly */ + if (OidIsValid(aggform->aggserialfn)) + { + Expr *serialfnexpr = NULL; + FmgrInfo serialfn; + build_aggregate_serialfn_expr(aggform->aggserialfn, + &serialfnexpr); + fmgr_info(aggform->aggserialfn, &serialfn); + fmgr_info_set_expr((Node *) serialfnexpr, &serialfn); - InitFunctionCallInfoData(*innerFcinfo, &info, 1, fcinfo->fncollation, - fcinfo->context, fcinfo->resultinfo); - fcSetArgExt(innerFcinfo, 0, box->value, box->valueNull); + LOCAL_FCINFO(serialfn_fcinfo, 2); + InitFunctionCallInfoData(*serialfn_fcinfo, + &serialfn, + 1, + InvalidOid, + (void *) fcinfo->context, NULL); - Datum result = FunctionCallInvoke(innerFcinfo); + fcSetArgExt(serialfn_fcinfo, 0, + MakeExpandedObjectReadOnly(box->value, box->valueNull, + box->transtypeLen), + false); - if (innerFcinfo->isnull) + result = FunctionCallInvoke(serialfn_fcinfo); + isNull = serialfn_fcinfo->isnull; + } + else + { + if (serializeToCString) + { + getTypeOutputInfo(transtype, &typoutput, &typIsVarlena); + } + else + { + getTypeBinaryOutputInfo(transtype, &typoutput, &typIsVarlena); + } + + fmgr_info(typoutput, &info); + + InitFunctionCallInfoData(*innerFcinfo, &info, 1, fcinfo->fncollation, + fcinfo->context, fcinfo->resultinfo); + fcSetArgExt(innerFcinfo, 0, box->value, box->valueNull); + + result = FunctionCallInvoke(innerFcinfo); + isNull = innerFcinfo->isnull; + } + + if (isNull) { PG_RETURN_NULL(); } @@ -674,7 +800,7 @@ worker_partial_agg_ffunc(PG_FUNCTION_ARGS) * return box */ Datum -coord_combine_agg_sfunc(PG_FUNCTION_ARGS) +combine_agg_sfunc_internal(PG_FUNCTION_ARGS, bool deserializeFromCString) { LOCAL_FCINFO(innerFcinfo, 3); FmgrInfo info; @@ -702,7 +828,7 @@ coord_combine_agg_sfunc(PG_FUNCTION_ARGS) "coord_combine_agg_sfunc expects an aggregate with COMBINEFUNC"))); } - if (aggform->aggtranstype == INTERNALOID) + if (deserializeFromCString && aggform->aggtranstype == INTERNALOID) { ereport(ERROR, (errmsg( @@ -728,19 +854,67 @@ coord_combine_agg_sfunc(PG_FUNCTION_ARGS) bool valueNull = PG_ARGISNULL(2); HeapTuple transtypetuple = GetTypeForm(box->transtype, &transtypeform); Oid ioparam = getTypeIOParam(transtypetuple); - Oid deserial = transtypeform->typinput; + Oid deserial = deserializeFromCString ? transtypeform->typinput : + transtypeform->typreceive; ReleaseSysCache(transtypetuple); - fmgr_info(deserial, &info); - if (valueNull && info.fn_strict) + bool strictDeserial = false; + if (OidIsValid(aggform->aggdeserialfn)) + { + strictDeserial = true; + } + else + { + fmgr_info(deserial, &info); + strictDeserial = info.fn_strict; + } + + if (valueNull && strictDeserial) { value = (Datum) 0; } + /* TODO: store function call info in box to avoid recurring initialization overhead */ + else if (OidIsValid(aggform->aggdeserialfn)) + { + Expr *deserialfnexpr = NULL; + FmgrInfo deserialfn; + build_aggregate_deserialfn_expr(aggform->aggdeserialfn, + &deserialfnexpr); + fmgr_info(aggform->aggdeserialfn, &deserialfn); + fmgr_info_set_expr((Node *) deserialfnexpr, &deserialfn); + + LOCAL_FCINFO(deserialfn_fcinfo, 2); + InitFunctionCallInfoData(*deserialfn_fcinfo, + &deserialfn, + 2, + InvalidOid, + (void *) fcinfo->context, NULL); + + + fcSetArgExt(deserialfn_fcinfo, 0, PointerGetDatum(PG_GETARG_BYTEA_P(2)), + valueNull); + fcSetArgExt(deserialfn_fcinfo, 1, PointerGetDatum(NULL), false); + + value = FunctionCallInvoke(deserialfn_fcinfo); + valueNull = deserialfn_fcinfo->isnull; + } else { InitFunctionCallInfoData(*innerFcinfo, &info, 3, fcinfo->fncollation, fcinfo->context, fcinfo->resultinfo); - fcSetArgExt(innerFcinfo, 0, PG_GETARG_DATUM(2), valueNull); + if (deserializeFromCString) + { + fcSetArgExt(innerFcinfo, 0, PG_GETARG_DATUM(2), valueNull); + } + else + { + StringInfo string = makeStringInfo(); + appendBinaryStringInfo(string, + VARDATA_ANY(PG_GETARG_DATUM(2)), + VARSIZE_ANY_EXHDR(PG_GETARG_DATUM(2))); + fcSetArgExt(innerFcinfo, 0, PointerGetDatum(string), valueNull); + } + fcSetArg(innerFcinfo, 1, ObjectIdGetDatum(ioparam)); fcSetArg(innerFcinfo, 2, Int32GetDatum(-1)); /* typmod */ @@ -788,7 +962,7 @@ coord_combine_agg_sfunc(PG_FUNCTION_ARGS) * return box.agg.ffunc(box.value) */ Datum -coord_combine_agg_ffunc(PG_FUNCTION_ARGS) +combine_agg_ffunc_internal(PG_FUNCTION_ARGS) { StypeBox *box = (StypeBox *) (PG_ARGISNULL(0) ? NULL : PG_GETARG_POINTER(0)); LOCAL_FCINFO(innerFcinfo, FUNC_MAX_ARGS); @@ -872,7 +1046,7 @@ TypecheckWorkerPartialAggArgType(FunctionCallInfo fcinfo, StypeBox *box) return false; } - Assert(list_length(aggref->args) == 2); + Assert(list_length(aggref->args) == 2 || list_length(aggref->args) == 3); TargetEntry *aggarg = list_nth(aggref->args, 1); bool argtypesNull; diff --git a/src/include/distributed/cimv.h b/src/include/distributed/cimv.h new file mode 100644 index 000000000..ddd37901d --- /dev/null +++ b/src/include/distributed/cimv.h @@ -0,0 +1,20 @@ +#ifndef CIMV_H +#define CIMV_H + +#include "postgres.h" +#include "nodes/plannodes.h" + +#define CITUS_INTERNAL_SCHEMA "citus_internal" +#define CITUS_NAMESPACE "citus" +#define MATERIALIZATION_TABLE_SUFFIX "mt" +#define LANDING_TABLE_SUFFIX "ld" +#define REFRESH_VIEW_SUFFIX "rv" + +extern bool ProcessCreateMaterializedViewStmt(const CreateTableAsStmt *stmt, const + char *query_string, PlannedStmt *pstmt); +extern void ProcessDropMaterializedViewStmt(DropStmt *stmt); +extern void ProcessDropViewStmt(DropStmt *stmt); +extern bool ProcessRefreshMaterializedViewStmt(RefreshMatViewStmt *stmt); +extern void RefreshCimv(Form_pg_cimv formCimv, bool skipData, bool isCreate); + +#endif diff --git a/src/include/distributed/metadata_cache.h b/src/include/distributed/metadata_cache.h index 7303e55fe..9edd5d1c0 100644 --- a/src/include/distributed/metadata_cache.h +++ b/src/include/distributed/metadata_cache.h @@ -203,6 +203,8 @@ extern Oid CitusCatalogNamespaceId(void); extern Oid DistColocationRelationId(void); extern Oid DistColocationConfigurationIndexId(void); extern Oid DistPartitionRelationId(void); +extern Oid PgCimvId(void); +extern Oid PgCimvIndexId(void); extern Oid DistShardRelationId(void); extern Oid DistPlacementRelationId(void); extern Oid DistNodeRelationId(void); diff --git a/src/include/distributed/metadata_utility.h b/src/include/distributed/metadata_utility.h index 5db5b0b69..ee2aa08e5 100644 --- a/src/include/distributed/metadata_utility.h +++ b/src/include/distributed/metadata_utility.h @@ -23,6 +23,7 @@ #include "catalog/objectaddress.h" #include "distributed/citus_nodes.h" #include "distributed/relay_utility.h" +#include "distributed/pg_cimv.h" #include "utils/acl.h" #include "utils/relcache.h" @@ -125,6 +126,9 @@ extern uint64 InsertShardPlacementRow(uint64 shardId, uint64 placementId, extern void InsertIntoPgDistPartition(Oid relationId, char distributionMethod, Var *distributionColumn, uint32 colocationId, char replicationModel); +extern void InsertIntoPgCimv(Form_pg_cimv cimv); +extern void DeletePgCimvRow(Oid userViewId); +extern Form_pg_cimv LookupCimvFromCatalog(Oid userViewId, bool missingOk); extern void DeletePartitionRow(Oid distributedRelationId); extern void DeleteShardRow(uint64 shardId); extern void UpdatePartitionShardPlacementStates(ShardPlacement *parentShardPlacement, diff --git a/src/include/distributed/pg_cimv.h b/src/include/distributed/pg_cimv.h new file mode 100644 index 000000000..83ff7b60f --- /dev/null +++ b/src/include/distributed/pg_cimv.h @@ -0,0 +1,51 @@ +/*------------------------------------------------------------------------- + * + * pg_cimv.h + * TODO + * + * Copyright (c) Citus Data, Inc. + * + *------------------------------------------------------------------------- + */ + +#ifndef PG_CIMV_H +#define PG_CIMV_H + +/* ---------------- + * pg_cimv definition. + * ---------------- + */ +typedef struct FormData_pg_cimv +{ + Oid userview; + Oid basetable; + Oid mattable; + Oid refreshview; + NameData triggerfnnamespace; + NameData triggerfnname; + Oid landingtable; + int64 jobid; +} FormData_pg_cimv; + +/* ---------------- + * FormData_pg_cimv corresponds to a pointer to a tuple with + * the format of pg_cimv relation. + * ---------------- + */ +typedef FormData_pg_cimv *Form_pg_cimv; + +/* ---------------- + * compiler constants for pg_cimv + * ---------------- + */ +#define Natts_pg_cimv 8 +#define Anum_pg_cimv_userview 1 +#define Anum_pg_cimv_basetable 2 +#define Anum_pg_cimv_mattable 3 +#define Anum_pg_cimv_refreshview 4 +#define Anum_pg_cimv_triggernamespace 5 +#define Anum_pg_cimv_triggername 6 +#define Anum_pg_cimv_landingtable 7 +#define Anum_pg_cimv_jobid 8 + +#endif /* PG_CIMV_H */ diff --git a/src/test/regress/expected/cimv.out b/src/test/regress/expected/cimv.out new file mode 100644 index 000000000..f5e148b10 --- /dev/null +++ b/src/test/regress/expected/cimv.out @@ -0,0 +1,1681 @@ +-- +-- CIMV +-- Tests for Citus Incremental Materialized Views +-- +\set VERBOSITY terse +SET citus.next_shard_id TO 400000; +CREATE SCHEMA cimv; +SET search_path TO cimv, public; +SET citus.shard_count TO 4; +CREATE TABLE events (a int, b int, c double precision, d timestamp, e bigint); +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(1, 10000) v; +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + min(b) AS min_b, + max(b) AS max_b, + avg(b) AS avg_b, + min(c) AS min_c, + max(c) AS max_c, + avg(c) AS avg_c, + min(e) AS min_e, + max(e) AS max_e, + avg(e) AS avg_e +FROM events +WHERE b > 10 +GROUP BY a, d_hour; +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | min_b | max_b | avg_b | min_c | max_c | avg_c | min_e | max_e | avg_e +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 20.00 | 90.00 | 55.00 | 6.67 | 663.33 | 335.00 | 20.00 | 1990.00 | 1005.00 + 0 | Wed Jan 01 21:00:00 2020 | 20.00 | 90.00 | 55.00 | 673.33 | 1330.00 | 1001.67 | 2020.00 | 3990.00 | 3005.00 + 0 | Wed Jan 01 22:00:00 2020 | 20.00 | 90.00 | 55.00 | 1340.00 | 1996.67 | 1668.33 | 4020.00 | 5990.00 | 5005.00 + 0 | Wed Jan 01 23:00:00 2020 | 20.00 | 90.00 | 55.00 | 2006.67 | 2663.33 | 2335.00 | 6020.00 | 7990.00 | 7005.00 + 0 | Thu Jan 02 00:00:00 2020 | 20.00 | 90.00 | 55.00 | 2673.33 | 3330.00 | 3001.67 | 8020.00 | 9990.00 | 9005.00 + 1 | Wed Jan 01 20:00:00 2020 | 11.00 | 91.00 | 51.00 | 3.67 | 663.67 | 333.67 | 11.00 | 1991.00 | 1001.00 + 1 | Wed Jan 01 21:00:00 2020 | 11.00 | 91.00 | 51.00 | 670.33 | 1330.33 | 1000.33 | 2011.00 | 3991.00 | 3001.00 + 1 | Wed Jan 01 22:00:00 2020 | 11.00 | 91.00 | 51.00 | 1337.00 | 1997.00 | 1667.00 | 4011.00 | 5991.00 | 5001.00 + 1 | Wed Jan 01 23:00:00 2020 | 11.00 | 91.00 | 51.00 | 2003.67 | 2663.67 | 2333.67 | 6011.00 | 7991.00 | 7001.00 + 1 | Thu Jan 02 00:00:00 2020 | 11.00 | 91.00 | 51.00 | 2670.33 | 3330.33 | 3000.33 | 8011.00 | 9991.00 | 9001.00 + 2 | Wed Jan 01 20:00:00 2020 | 12.00 | 92.00 | 52.00 | 4.00 | 664.00 | 334.00 | 12.00 | 1992.00 | 1002.00 + 2 | Wed Jan 01 21:00:00 2020 | 12.00 | 92.00 | 52.00 | 670.67 | 1330.67 | 1000.67 | 2012.00 | 3992.00 | 3002.00 + 2 | Wed Jan 01 22:00:00 2020 | 12.00 | 92.00 | 52.00 | 1337.33 | 1997.33 | 1667.33 | 4012.00 | 5992.00 | 5002.00 + 2 | Wed Jan 01 23:00:00 2020 | 12.00 | 92.00 | 52.00 | 2004.00 | 2664.00 | 2334.00 | 6012.00 | 7992.00 | 7002.00 + 2 | Thu Jan 02 00:00:00 2020 | 12.00 | 92.00 | 52.00 | 2670.67 | 3330.67 | 3000.67 | 8012.00 | 9992.00 | 9002.00 + 3 | Wed Jan 01 20:00:00 2020 | 13.00 | 93.00 | 53.00 | 4.33 | 664.33 | 334.33 | 13.00 | 1993.00 | 1003.00 + 3 | Wed Jan 01 21:00:00 2020 | 13.00 | 93.00 | 53.00 | 671.00 | 1331.00 | 1001.00 | 2013.00 | 3993.00 | 3003.00 + 3 | Wed Jan 01 22:00:00 2020 | 13.00 | 93.00 | 53.00 | 1337.67 | 1997.67 | 1667.67 | 4013.00 | 5993.00 | 5003.00 + 3 | Wed Jan 01 23:00:00 2020 | 13.00 | 93.00 | 53.00 | 2004.33 | 2664.33 | 2334.33 | 6013.00 | 7993.00 | 7003.00 + 3 | Thu Jan 02 00:00:00 2020 | 13.00 | 93.00 | 53.00 | 2671.00 | 3331.00 | 3001.00 | 8013.00 | 9993.00 | 9003.00 + 4 | Wed Jan 01 20:00:00 2020 | 14.00 | 94.00 | 54.00 | 4.67 | 664.67 | 334.67 | 14.00 | 1994.00 | 1004.00 + 4 | Wed Jan 01 21:00:00 2020 | 14.00 | 94.00 | 54.00 | 671.33 | 1331.33 | 1001.33 | 2014.00 | 3994.00 | 3004.00 + 4 | Wed Jan 01 22:00:00 2020 | 14.00 | 94.00 | 54.00 | 1338.00 | 1998.00 | 1668.00 | 4014.00 | 5994.00 | 5004.00 + 4 | Wed Jan 01 23:00:00 2020 | 14.00 | 94.00 | 54.00 | 2004.67 | 2664.67 | 2334.67 | 6014.00 | 7994.00 | 7004.00 + 4 | Thu Jan 02 00:00:00 2020 | 14.00 | 94.00 | 54.00 | 2671.33 | 3331.33 | 3001.33 | 8014.00 | 9994.00 | 9004.00 + 5 | Wed Jan 01 20:00:00 2020 | 15.00 | 95.00 | 55.00 | 5.00 | 665.00 | 335.00 | 15.00 | 1995.00 | 1005.00 + 5 | Wed Jan 01 21:00:00 2020 | 15.00 | 95.00 | 55.00 | 671.67 | 1331.67 | 1001.67 | 2015.00 | 3995.00 | 3005.00 + 5 | Wed Jan 01 22:00:00 2020 | 15.00 | 95.00 | 55.00 | 1338.33 | 1998.33 | 1668.33 | 4015.00 | 5995.00 | 5005.00 + 5 | Wed Jan 01 23:00:00 2020 | 15.00 | 95.00 | 55.00 | 2005.00 | 2665.00 | 2335.00 | 6015.00 | 7995.00 | 7005.00 + 5 | Thu Jan 02 00:00:00 2020 | 15.00 | 95.00 | 55.00 | 2671.67 | 3331.67 | 3001.67 | 8015.00 | 9995.00 | 9005.00 + 6 | Wed Jan 01 20:00:00 2020 | 16.00 | 96.00 | 56.00 | 5.33 | 665.33 | 335.33 | 16.00 | 1996.00 | 1006.00 + 6 | Wed Jan 01 21:00:00 2020 | 16.00 | 96.00 | 56.00 | 672.00 | 1332.00 | 1002.00 | 2016.00 | 3996.00 | 3006.00 + 6 | Wed Jan 01 22:00:00 2020 | 16.00 | 96.00 | 56.00 | 1338.67 | 1998.67 | 1668.67 | 4016.00 | 5996.00 | 5006.00 + 6 | Wed Jan 01 23:00:00 2020 | 16.00 | 96.00 | 56.00 | 2005.33 | 2665.33 | 2335.33 | 6016.00 | 7996.00 | 7006.00 + 6 | Thu Jan 02 00:00:00 2020 | 16.00 | 96.00 | 56.00 | 2672.00 | 3332.00 | 3002.00 | 8016.00 | 9996.00 | 9006.00 + 7 | Wed Jan 01 20:00:00 2020 | 17.00 | 97.00 | 57.00 | 5.67 | 665.67 | 335.67 | 17.00 | 1997.00 | 1007.00 + 7 | Wed Jan 01 21:00:00 2020 | 17.00 | 97.00 | 57.00 | 672.33 | 1332.33 | 1002.33 | 2017.00 | 3997.00 | 3007.00 + 7 | Wed Jan 01 22:00:00 2020 | 17.00 | 97.00 | 57.00 | 1339.00 | 1999.00 | 1669.00 | 4017.00 | 5997.00 | 5007.00 + 7 | Wed Jan 01 23:00:00 2020 | 17.00 | 97.00 | 57.00 | 2005.67 | 2665.67 | 2335.67 | 6017.00 | 7997.00 | 7007.00 + 7 | Thu Jan 02 00:00:00 2020 | 17.00 | 97.00 | 57.00 | 2672.33 | 3332.33 | 3002.33 | 8017.00 | 9997.00 | 9007.00 + 8 | Wed Jan 01 20:00:00 2020 | 18.00 | 98.00 | 58.00 | 6.00 | 666.00 | 336.00 | 18.00 | 1998.00 | 1008.00 + 8 | Wed Jan 01 21:00:00 2020 | 18.00 | 98.00 | 58.00 | 672.67 | 1332.67 | 1002.67 | 2018.00 | 3998.00 | 3008.00 + 8 | Wed Jan 01 22:00:00 2020 | 18.00 | 98.00 | 58.00 | 1339.33 | 1999.33 | 1669.33 | 4018.00 | 5998.00 | 5008.00 + 8 | Wed Jan 01 23:00:00 2020 | 18.00 | 98.00 | 58.00 | 2006.00 | 2666.00 | 2336.00 | 6018.00 | 7998.00 | 7008.00 + 8 | Thu Jan 02 00:00:00 2020 | 18.00 | 98.00 | 58.00 | 2672.67 | 3332.67 | 3002.67 | 8018.00 | 9998.00 | 9008.00 + 9 | Wed Jan 01 20:00:00 2020 | 19.00 | 99.00 | 59.00 | 6.33 | 666.33 | 336.33 | 19.00 | 1999.00 | 1009.00 + 9 | Wed Jan 01 21:00:00 2020 | 19.00 | 99.00 | 59.00 | 673.00 | 1333.00 | 1003.00 | 2019.00 | 3999.00 | 3009.00 + 9 | Wed Jan 01 22:00:00 2020 | 19.00 | 99.00 | 59.00 | 1339.67 | 1999.67 | 1669.67 | 4019.00 | 5999.00 | 5009.00 + 9 | Wed Jan 01 23:00:00 2020 | 19.00 | 99.00 | 59.00 | 2006.33 | 2666.33 | 2336.33 | 6019.00 | 7999.00 | 7009.00 + 9 | Thu Jan 02 00:00:00 2020 | 19.00 | 99.00 | 59.00 | 2673.00 | 3333.00 | 3003.00 | 8019.00 | 9999.00 | 9009.00 +(50 rows) + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(10000, 11000) v; +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | min_b | max_b | avg_b | min_c | max_c | avg_c | min_e | max_e | avg_e +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 20.00 | 90.00 | 55.00 | 6.67 | 663.33 | 335.00 | 20.00 | 1990.00 | 1005.00 + 0 | Wed Jan 01 21:00:00 2020 | 20.00 | 90.00 | 55.00 | 673.33 | 1330.00 | 1001.67 | 2020.00 | 3990.00 | 3005.00 + 0 | Wed Jan 01 22:00:00 2020 | 20.00 | 90.00 | 55.00 | 1340.00 | 1996.67 | 1668.33 | 4020.00 | 5990.00 | 5005.00 + 0 | Wed Jan 01 23:00:00 2020 | 20.00 | 90.00 | 55.00 | 2006.67 | 2663.33 | 2335.00 | 6020.00 | 7990.00 | 7005.00 + 0 | Thu Jan 02 00:00:00 2020 | 20.00 | 90.00 | 55.00 | 2673.33 | 3330.00 | 3001.67 | 8020.00 | 9990.00 | 9005.00 + 0 | Thu Jan 02 01:00:00 2020 | 20.00 | 90.00 | 55.00 | 3340.00 | 3663.33 | 3501.67 | 10020.00 | 10990.00 | 10505.00 + 1 | Wed Jan 01 20:00:00 2020 | 11.00 | 91.00 | 51.00 | 3.67 | 663.67 | 333.67 | 11.00 | 1991.00 | 1001.00 + 1 | Wed Jan 01 21:00:00 2020 | 11.00 | 91.00 | 51.00 | 670.33 | 1330.33 | 1000.33 | 2011.00 | 3991.00 | 3001.00 + 1 | Wed Jan 01 22:00:00 2020 | 11.00 | 91.00 | 51.00 | 1337.00 | 1997.00 | 1667.00 | 4011.00 | 5991.00 | 5001.00 + 1 | Wed Jan 01 23:00:00 2020 | 11.00 | 91.00 | 51.00 | 2003.67 | 2663.67 | 2333.67 | 6011.00 | 7991.00 | 7001.00 + 1 | Thu Jan 02 00:00:00 2020 | 11.00 | 91.00 | 51.00 | 2670.33 | 3330.33 | 3000.33 | 8011.00 | 9991.00 | 9001.00 + 1 | Thu Jan 02 01:00:00 2020 | 11.00 | 91.00 | 51.00 | 3337.00 | 3663.67 | 3500.33 | 10011.00 | 10991.00 | 10501.00 + 2 | Wed Jan 01 20:00:00 2020 | 12.00 | 92.00 | 52.00 | 4.00 | 664.00 | 334.00 | 12.00 | 1992.00 | 1002.00 + 2 | Wed Jan 01 21:00:00 2020 | 12.00 | 92.00 | 52.00 | 670.67 | 1330.67 | 1000.67 | 2012.00 | 3992.00 | 3002.00 + 2 | Wed Jan 01 22:00:00 2020 | 12.00 | 92.00 | 52.00 | 1337.33 | 1997.33 | 1667.33 | 4012.00 | 5992.00 | 5002.00 + 2 | Wed Jan 01 23:00:00 2020 | 12.00 | 92.00 | 52.00 | 2004.00 | 2664.00 | 2334.00 | 6012.00 | 7992.00 | 7002.00 + 2 | Thu Jan 02 00:00:00 2020 | 12.00 | 92.00 | 52.00 | 2670.67 | 3330.67 | 3000.67 | 8012.00 | 9992.00 | 9002.00 + 2 | Thu Jan 02 01:00:00 2020 | 12.00 | 92.00 | 52.00 | 3337.33 | 3664.00 | 3500.67 | 10012.00 | 10992.00 | 10502.00 + 3 | Wed Jan 01 20:00:00 2020 | 13.00 | 93.00 | 53.00 | 4.33 | 664.33 | 334.33 | 13.00 | 1993.00 | 1003.00 + 3 | Wed Jan 01 21:00:00 2020 | 13.00 | 93.00 | 53.00 | 671.00 | 1331.00 | 1001.00 | 2013.00 | 3993.00 | 3003.00 + 3 | Wed Jan 01 22:00:00 2020 | 13.00 | 93.00 | 53.00 | 1337.67 | 1997.67 | 1667.67 | 4013.00 | 5993.00 | 5003.00 + 3 | Wed Jan 01 23:00:00 2020 | 13.00 | 93.00 | 53.00 | 2004.33 | 2664.33 | 2334.33 | 6013.00 | 7993.00 | 7003.00 + 3 | Thu Jan 02 00:00:00 2020 | 13.00 | 93.00 | 53.00 | 2671.00 | 3331.00 | 3001.00 | 8013.00 | 9993.00 | 9003.00 + 3 | Thu Jan 02 01:00:00 2020 | 13.00 | 93.00 | 53.00 | 3337.67 | 3664.33 | 3501.00 | 10013.00 | 10993.00 | 10503.00 + 4 | Wed Jan 01 20:00:00 2020 | 14.00 | 94.00 | 54.00 | 4.67 | 664.67 | 334.67 | 14.00 | 1994.00 | 1004.00 + 4 | Wed Jan 01 21:00:00 2020 | 14.00 | 94.00 | 54.00 | 671.33 | 1331.33 | 1001.33 | 2014.00 | 3994.00 | 3004.00 + 4 | Wed Jan 01 22:00:00 2020 | 14.00 | 94.00 | 54.00 | 1338.00 | 1998.00 | 1668.00 | 4014.00 | 5994.00 | 5004.00 + 4 | Wed Jan 01 23:00:00 2020 | 14.00 | 94.00 | 54.00 | 2004.67 | 2664.67 | 2334.67 | 6014.00 | 7994.00 | 7004.00 + 4 | Thu Jan 02 00:00:00 2020 | 14.00 | 94.00 | 54.00 | 2671.33 | 3331.33 | 3001.33 | 8014.00 | 9994.00 | 9004.00 + 4 | Thu Jan 02 01:00:00 2020 | 14.00 | 94.00 | 54.00 | 3338.00 | 3664.67 | 3501.33 | 10014.00 | 10994.00 | 10504.00 + 5 | Wed Jan 01 20:00:00 2020 | 15.00 | 95.00 | 55.00 | 5.00 | 665.00 | 335.00 | 15.00 | 1995.00 | 1005.00 + 5 | Wed Jan 01 21:00:00 2020 | 15.00 | 95.00 | 55.00 | 671.67 | 1331.67 | 1001.67 | 2015.00 | 3995.00 | 3005.00 + 5 | Wed Jan 01 22:00:00 2020 | 15.00 | 95.00 | 55.00 | 1338.33 | 1998.33 | 1668.33 | 4015.00 | 5995.00 | 5005.00 + 5 | Wed Jan 01 23:00:00 2020 | 15.00 | 95.00 | 55.00 | 2005.00 | 2665.00 | 2335.00 | 6015.00 | 7995.00 | 7005.00 + 5 | Thu Jan 02 00:00:00 2020 | 15.00 | 95.00 | 55.00 | 2671.67 | 3331.67 | 3001.67 | 8015.00 | 9995.00 | 9005.00 + 5 | Thu Jan 02 01:00:00 2020 | 15.00 | 95.00 | 55.00 | 3338.33 | 3665.00 | 3501.67 | 10015.00 | 10995.00 | 10505.00 + 6 | Wed Jan 01 20:00:00 2020 | 16.00 | 96.00 | 56.00 | 5.33 | 665.33 | 335.33 | 16.00 | 1996.00 | 1006.00 + 6 | Wed Jan 01 21:00:00 2020 | 16.00 | 96.00 | 56.00 | 672.00 | 1332.00 | 1002.00 | 2016.00 | 3996.00 | 3006.00 + 6 | Wed Jan 01 22:00:00 2020 | 16.00 | 96.00 | 56.00 | 1338.67 | 1998.67 | 1668.67 | 4016.00 | 5996.00 | 5006.00 + 6 | Wed Jan 01 23:00:00 2020 | 16.00 | 96.00 | 56.00 | 2005.33 | 2665.33 | 2335.33 | 6016.00 | 7996.00 | 7006.00 + 6 | Thu Jan 02 00:00:00 2020 | 16.00 | 96.00 | 56.00 | 2672.00 | 3332.00 | 3002.00 | 8016.00 | 9996.00 | 9006.00 + 6 | Thu Jan 02 01:00:00 2020 | 16.00 | 96.00 | 56.00 | 3338.67 | 3665.33 | 3502.00 | 10016.00 | 10996.00 | 10506.00 + 7 | Wed Jan 01 20:00:00 2020 | 17.00 | 97.00 | 57.00 | 5.67 | 665.67 | 335.67 | 17.00 | 1997.00 | 1007.00 + 7 | Wed Jan 01 21:00:00 2020 | 17.00 | 97.00 | 57.00 | 672.33 | 1332.33 | 1002.33 | 2017.00 | 3997.00 | 3007.00 + 7 | Wed Jan 01 22:00:00 2020 | 17.00 | 97.00 | 57.00 | 1339.00 | 1999.00 | 1669.00 | 4017.00 | 5997.00 | 5007.00 + 7 | Wed Jan 01 23:00:00 2020 | 17.00 | 97.00 | 57.00 | 2005.67 | 2665.67 | 2335.67 | 6017.00 | 7997.00 | 7007.00 + 7 | Thu Jan 02 00:00:00 2020 | 17.00 | 97.00 | 57.00 | 2672.33 | 3332.33 | 3002.33 | 8017.00 | 9997.00 | 9007.00 + 7 | Thu Jan 02 01:00:00 2020 | 17.00 | 97.00 | 57.00 | 3339.00 | 3665.67 | 3502.33 | 10017.00 | 10997.00 | 10507.00 + 8 | Wed Jan 01 20:00:00 2020 | 18.00 | 98.00 | 58.00 | 6.00 | 666.00 | 336.00 | 18.00 | 1998.00 | 1008.00 + 8 | Wed Jan 01 21:00:00 2020 | 18.00 | 98.00 | 58.00 | 672.67 | 1332.67 | 1002.67 | 2018.00 | 3998.00 | 3008.00 + 8 | Wed Jan 01 22:00:00 2020 | 18.00 | 98.00 | 58.00 | 1339.33 | 1999.33 | 1669.33 | 4018.00 | 5998.00 | 5008.00 + 8 | Wed Jan 01 23:00:00 2020 | 18.00 | 98.00 | 58.00 | 2006.00 | 2666.00 | 2336.00 | 6018.00 | 7998.00 | 7008.00 + 8 | Thu Jan 02 00:00:00 2020 | 18.00 | 98.00 | 58.00 | 2672.67 | 3332.67 | 3002.67 | 8018.00 | 9998.00 | 9008.00 + 8 | Thu Jan 02 01:00:00 2020 | 18.00 | 98.00 | 58.00 | 3339.33 | 3666.00 | 3502.67 | 10018.00 | 10998.00 | 10508.00 + 9 | Wed Jan 01 20:00:00 2020 | 19.00 | 99.00 | 59.00 | 6.33 | 666.33 | 336.33 | 19.00 | 1999.00 | 1009.00 + 9 | Wed Jan 01 21:00:00 2020 | 19.00 | 99.00 | 59.00 | 673.00 | 1333.00 | 1003.00 | 2019.00 | 3999.00 | 3009.00 + 9 | Wed Jan 01 22:00:00 2020 | 19.00 | 99.00 | 59.00 | 1339.67 | 1999.67 | 1669.67 | 4019.00 | 5999.00 | 5009.00 + 9 | Wed Jan 01 23:00:00 2020 | 19.00 | 99.00 | 59.00 | 2006.33 | 2666.33 | 2336.33 | 6019.00 | 7999.00 | 7009.00 + 9 | Thu Jan 02 00:00:00 2020 | 19.00 | 99.00 | 59.00 | 2673.00 | 3333.00 | 3003.00 | 8019.00 | 9999.00 | 9009.00 + 9 | Thu Jan 02 01:00:00 2020 | 19.00 | 99.00 | 59.00 | 3339.67 | 3666.33 | 3503.00 | 10019.00 | 10999.00 | 10509.00 +(60 rows) + +DELETE FROM events WHERE b < 100; +ERROR: MATERIALIZED VIEW 'mv' on table 'events' does not support UPDATE/DELETE +DROP VIEW mv; +ERROR: DROP VIEW not supported for mv +DROP MATERIALIZED VIEW mv; +NOTICE: drop cascades to 4 other objects +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 55.00 + 0 | Wed Jan 01 21:00:00 2020 | 55.00 + 0 | Wed Jan 01 22:00:00 2020 | 55.00 + 0 | Wed Jan 01 23:00:00 2020 | 55.00 + 0 | Thu Jan 02 00:00:00 2020 | 55.00 + 0 | Thu Jan 02 01:00:00 2020 | 55.00 + 1 | Wed Jan 01 20:00:00 2020 | 51.00 + 1 | Wed Jan 01 21:00:00 2020 | 51.00 + 1 | Wed Jan 01 22:00:00 2020 | 51.00 + 1 | Wed Jan 01 23:00:00 2020 | 51.00 + 1 | Thu Jan 02 00:00:00 2020 | 51.00 + 1 | Thu Jan 02 01:00:00 2020 | 51.00 + 2 | Wed Jan 01 20:00:00 2020 | 52.00 + 2 | Wed Jan 01 21:00:00 2020 | 52.00 + 2 | Wed Jan 01 22:00:00 2020 | 52.00 + 2 | Wed Jan 01 23:00:00 2020 | 52.00 + 2 | Thu Jan 02 00:00:00 2020 | 52.00 + 2 | Thu Jan 02 01:00:00 2020 | 52.00 + 3 | Wed Jan 01 20:00:00 2020 | 53.00 + 3 | Wed Jan 01 21:00:00 2020 | 53.00 + 3 | Wed Jan 01 22:00:00 2020 | 53.00 + 3 | Wed Jan 01 23:00:00 2020 | 53.00 + 3 | Thu Jan 02 00:00:00 2020 | 53.00 + 3 | Thu Jan 02 01:00:00 2020 | 53.00 + 4 | Wed Jan 01 20:00:00 2020 | 54.00 + 4 | Wed Jan 01 21:00:00 2020 | 54.00 + 4 | Wed Jan 01 22:00:00 2020 | 54.00 + 4 | Wed Jan 01 23:00:00 2020 | 54.00 + 4 | Thu Jan 02 00:00:00 2020 | 54.00 + 4 | Thu Jan 02 01:00:00 2020 | 54.00 + 5 | Wed Jan 01 20:00:00 2020 | 55.00 + 5 | Wed Jan 01 21:00:00 2020 | 55.00 + 5 | Wed Jan 01 22:00:00 2020 | 55.00 + 5 | Wed Jan 01 23:00:00 2020 | 55.00 + 5 | Thu Jan 02 00:00:00 2020 | 55.00 + 5 | Thu Jan 02 01:00:00 2020 | 55.00 + 6 | Wed Jan 01 20:00:00 2020 | 56.00 + 6 | Wed Jan 01 21:00:00 2020 | 56.00 + 6 | Wed Jan 01 22:00:00 2020 | 56.00 + 6 | Wed Jan 01 23:00:00 2020 | 56.00 + 6 | Thu Jan 02 00:00:00 2020 | 56.00 + 6 | Thu Jan 02 01:00:00 2020 | 56.00 + 7 | Wed Jan 01 20:00:00 2020 | 57.00 + 7 | Wed Jan 01 21:00:00 2020 | 57.00 + 7 | Wed Jan 01 22:00:00 2020 | 57.00 + 7 | Wed Jan 01 23:00:00 2020 | 57.00 + 7 | Thu Jan 02 00:00:00 2020 | 57.00 + 7 | Thu Jan 02 01:00:00 2020 | 57.00 + 8 | Wed Jan 01 20:00:00 2020 | 58.00 + 8 | Wed Jan 01 21:00:00 2020 | 58.00 + 8 | Wed Jan 01 22:00:00 2020 | 58.00 + 8 | Wed Jan 01 23:00:00 2020 | 58.00 + 8 | Thu Jan 02 00:00:00 2020 | 58.00 + 8 | Thu Jan 02 01:00:00 2020 | 58.00 + 9 | Wed Jan 01 20:00:00 2020 | 59.00 + 9 | Wed Jan 01 21:00:00 2020 | 59.00 + 9 | Wed Jan 01 22:00:00 2020 | 59.00 + 9 | Wed Jan 01 23:00:00 2020 | 59.00 + 9 | Thu Jan 02 00:00:00 2020 | 59.00 + 9 | Thu Jan 02 01:00:00 2020 | 59.00 +(60 rows) + +DELETE FROM events WHERE b < 20; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 55.00 + 0 | Wed Jan 01 21:00:00 2020 | 55.00 + 0 | Wed Jan 01 22:00:00 2020 | 55.00 + 0 | Wed Jan 01 23:00:00 2020 | 55.00 + 0 | Thu Jan 02 00:00:00 2020 | 55.00 + 0 | Thu Jan 02 01:00:00 2020 | 55.00 + 1 | Wed Jan 01 20:00:00 2020 | 56.00 + 1 | Wed Jan 01 21:00:00 2020 | 56.00 + 1 | Wed Jan 01 22:00:00 2020 | 56.00 + 1 | Wed Jan 01 23:00:00 2020 | 56.00 + 1 | Thu Jan 02 00:00:00 2020 | 56.00 + 1 | Thu Jan 02 01:00:00 2020 | 56.00 + 2 | Wed Jan 01 20:00:00 2020 | 57.00 + 2 | Wed Jan 01 21:00:00 2020 | 57.00 + 2 | Wed Jan 01 22:00:00 2020 | 57.00 + 2 | Wed Jan 01 23:00:00 2020 | 57.00 + 2 | Thu Jan 02 00:00:00 2020 | 57.00 + 2 | Thu Jan 02 01:00:00 2020 | 57.00 + 3 | Wed Jan 01 20:00:00 2020 | 58.00 + 3 | Wed Jan 01 21:00:00 2020 | 58.00 + 3 | Wed Jan 01 22:00:00 2020 | 58.00 + 3 | Wed Jan 01 23:00:00 2020 | 58.00 + 3 | Thu Jan 02 00:00:00 2020 | 58.00 + 3 | Thu Jan 02 01:00:00 2020 | 58.00 + 4 | Wed Jan 01 20:00:00 2020 | 59.00 + 4 | Wed Jan 01 21:00:00 2020 | 59.00 + 4 | Wed Jan 01 22:00:00 2020 | 59.00 + 4 | Wed Jan 01 23:00:00 2020 | 59.00 + 4 | Thu Jan 02 00:00:00 2020 | 59.00 + 4 | Thu Jan 02 01:00:00 2020 | 59.00 + 5 | Wed Jan 01 20:00:00 2020 | 60.00 + 5 | Wed Jan 01 21:00:00 2020 | 60.00 + 5 | Wed Jan 01 22:00:00 2020 | 60.00 + 5 | Wed Jan 01 23:00:00 2020 | 60.00 + 5 | Thu Jan 02 00:00:00 2020 | 60.00 + 5 | Thu Jan 02 01:00:00 2020 | 60.00 + 6 | Wed Jan 01 20:00:00 2020 | 61.00 + 6 | Wed Jan 01 21:00:00 2020 | 61.00 + 6 | Wed Jan 01 22:00:00 2020 | 61.00 + 6 | Wed Jan 01 23:00:00 2020 | 61.00 + 6 | Thu Jan 02 00:00:00 2020 | 61.00 + 6 | Thu Jan 02 01:00:00 2020 | 61.00 + 7 | Wed Jan 01 20:00:00 2020 | 62.00 + 7 | Wed Jan 01 21:00:00 2020 | 62.00 + 7 | Wed Jan 01 22:00:00 2020 | 62.00 + 7 | Wed Jan 01 23:00:00 2020 | 62.00 + 7 | Thu Jan 02 00:00:00 2020 | 62.00 + 7 | Thu Jan 02 01:00:00 2020 | 62.00 + 8 | Wed Jan 01 20:00:00 2020 | 63.00 + 8 | Wed Jan 01 21:00:00 2020 | 63.00 + 8 | Wed Jan 01 22:00:00 2020 | 63.00 + 8 | Wed Jan 01 23:00:00 2020 | 63.00 + 8 | Thu Jan 02 00:00:00 2020 | 63.00 + 8 | Thu Jan 02 01:00:00 2020 | 63.00 + 9 | Wed Jan 01 20:00:00 2020 | 64.00 + 9 | Wed Jan 01 21:00:00 2020 | 64.00 + 9 | Wed Jan 01 22:00:00 2020 | 64.00 + 9 | Wed Jan 01 23:00:00 2020 | 64.00 + 9 | Thu Jan 02 00:00:00 2020 | 64.00 + 9 | Thu Jan 02 01:00:00 2020 | 64.00 +(60 rows) + +UPDATE events SET b = b + b; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 110.00 + 0 | Wed Jan 01 21:00:00 2020 | 110.00 + 0 | Wed Jan 01 22:00:00 2020 | 110.00 + 0 | Wed Jan 01 23:00:00 2020 | 110.00 + 0 | Thu Jan 02 00:00:00 2020 | 110.00 + 0 | Thu Jan 02 01:00:00 2020 | 110.00 + 1 | Wed Jan 01 20:00:00 2020 | 112.00 + 1 | Wed Jan 01 21:00:00 2020 | 112.00 + 1 | Wed Jan 01 22:00:00 2020 | 112.00 + 1 | Wed Jan 01 23:00:00 2020 | 112.00 + 1 | Thu Jan 02 00:00:00 2020 | 112.00 + 1 | Thu Jan 02 01:00:00 2020 | 112.00 + 2 | Wed Jan 01 20:00:00 2020 | 114.00 + 2 | Wed Jan 01 21:00:00 2020 | 114.00 + 2 | Wed Jan 01 22:00:00 2020 | 114.00 + 2 | Wed Jan 01 23:00:00 2020 | 114.00 + 2 | Thu Jan 02 00:00:00 2020 | 114.00 + 2 | Thu Jan 02 01:00:00 2020 | 114.00 + 3 | Wed Jan 01 20:00:00 2020 | 116.00 + 3 | Wed Jan 01 21:00:00 2020 | 116.00 + 3 | Wed Jan 01 22:00:00 2020 | 116.00 + 3 | Wed Jan 01 23:00:00 2020 | 116.00 + 3 | Thu Jan 02 00:00:00 2020 | 116.00 + 3 | Thu Jan 02 01:00:00 2020 | 116.00 + 4 | Wed Jan 01 20:00:00 2020 | 118.00 + 4 | Wed Jan 01 21:00:00 2020 | 118.00 + 4 | Wed Jan 01 22:00:00 2020 | 118.00 + 4 | Wed Jan 01 23:00:00 2020 | 118.00 + 4 | Thu Jan 02 00:00:00 2020 | 118.00 + 4 | Thu Jan 02 01:00:00 2020 | 118.00 + 5 | Wed Jan 01 20:00:00 2020 | 120.00 + 5 | Wed Jan 01 21:00:00 2020 | 120.00 + 5 | Wed Jan 01 22:00:00 2020 | 120.00 + 5 | Wed Jan 01 23:00:00 2020 | 120.00 + 5 | Thu Jan 02 00:00:00 2020 | 120.00 + 5 | Thu Jan 02 01:00:00 2020 | 120.00 + 6 | Wed Jan 01 20:00:00 2020 | 122.00 + 6 | Wed Jan 01 21:00:00 2020 | 122.00 + 6 | Wed Jan 01 22:00:00 2020 | 122.00 + 6 | Wed Jan 01 23:00:00 2020 | 122.00 + 6 | Thu Jan 02 00:00:00 2020 | 122.00 + 6 | Thu Jan 02 01:00:00 2020 | 122.00 + 7 | Wed Jan 01 20:00:00 2020 | 124.00 + 7 | Wed Jan 01 21:00:00 2020 | 124.00 + 7 | Wed Jan 01 22:00:00 2020 | 124.00 + 7 | Wed Jan 01 23:00:00 2020 | 124.00 + 7 | Thu Jan 02 00:00:00 2020 | 124.00 + 7 | Thu Jan 02 01:00:00 2020 | 124.00 + 8 | Wed Jan 01 20:00:00 2020 | 126.00 + 8 | Wed Jan 01 21:00:00 2020 | 126.00 + 8 | Wed Jan 01 22:00:00 2020 | 126.00 + 8 | Wed Jan 01 23:00:00 2020 | 126.00 + 8 | Thu Jan 02 00:00:00 2020 | 126.00 + 8 | Thu Jan 02 01:00:00 2020 | 126.00 + 9 | Wed Jan 01 20:00:00 2020 | 128.00 + 9 | Wed Jan 01 21:00:00 2020 | 128.00 + 9 | Wed Jan 01 22:00:00 2020 | 128.00 + 9 | Wed Jan 01 23:00:00 2020 | 128.00 + 9 | Thu Jan 02 00:00:00 2020 | 128.00 + 9 | Thu Jan 02 01:00:00 2020 | 128.00 +(60 rows) + +DROP MATERIALIZED VIEW mv; +NOTICE: drop cascades to 4 other objects +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 110.00 + 0 | Wed Jan 01 21:00:00 2020 | 110.00 + 0 | Wed Jan 01 22:00:00 2020 | 110.00 + 0 | Wed Jan 01 23:00:00 2020 | 110.00 + 0 | Thu Jan 02 00:00:00 2020 | 110.00 + 0 | Thu Jan 02 01:00:00 2020 | 110.00 + 1 | Wed Jan 01 20:00:00 2020 | 112.00 + 1 | Wed Jan 01 21:00:00 2020 | 112.00 + 1 | Wed Jan 01 22:00:00 2020 | 112.00 + 1 | Wed Jan 01 23:00:00 2020 | 112.00 + 1 | Thu Jan 02 00:00:00 2020 | 112.00 + 1 | Thu Jan 02 01:00:00 2020 | 112.00 + 2 | Wed Jan 01 20:00:00 2020 | 114.00 + 2 | Wed Jan 01 21:00:00 2020 | 114.00 + 2 | Wed Jan 01 22:00:00 2020 | 114.00 + 2 | Wed Jan 01 23:00:00 2020 | 114.00 + 2 | Thu Jan 02 00:00:00 2020 | 114.00 + 2 | Thu Jan 02 01:00:00 2020 | 114.00 + 3 | Wed Jan 01 20:00:00 2020 | 116.00 + 3 | Wed Jan 01 21:00:00 2020 | 116.00 + 3 | Wed Jan 01 22:00:00 2020 | 116.00 + 3 | Wed Jan 01 23:00:00 2020 | 116.00 + 3 | Thu Jan 02 00:00:00 2020 | 116.00 + 3 | Thu Jan 02 01:00:00 2020 | 116.00 + 4 | Wed Jan 01 20:00:00 2020 | 118.00 + 4 | Wed Jan 01 21:00:00 2020 | 118.00 + 4 | Wed Jan 01 22:00:00 2020 | 118.00 + 4 | Wed Jan 01 23:00:00 2020 | 118.00 + 4 | Thu Jan 02 00:00:00 2020 | 118.00 + 4 | Thu Jan 02 01:00:00 2020 | 118.00 + 5 | Wed Jan 01 20:00:00 2020 | 120.00 + 5 | Wed Jan 01 21:00:00 2020 | 120.00 + 5 | Wed Jan 01 22:00:00 2020 | 120.00 + 5 | Wed Jan 01 23:00:00 2020 | 120.00 + 5 | Thu Jan 02 00:00:00 2020 | 120.00 + 5 | Thu Jan 02 01:00:00 2020 | 120.00 + 6 | Wed Jan 01 20:00:00 2020 | 122.00 + 6 | Wed Jan 01 21:00:00 2020 | 122.00 + 6 | Wed Jan 01 22:00:00 2020 | 122.00 + 6 | Wed Jan 01 23:00:00 2020 | 122.00 + 6 | Thu Jan 02 00:00:00 2020 | 122.00 + 6 | Thu Jan 02 01:00:00 2020 | 122.00 + 7 | Wed Jan 01 20:00:00 2020 | 124.00 + 7 | Wed Jan 01 21:00:00 2020 | 124.00 + 7 | Wed Jan 01 22:00:00 2020 | 124.00 + 7 | Wed Jan 01 23:00:00 2020 | 124.00 + 7 | Thu Jan 02 00:00:00 2020 | 124.00 + 7 | Thu Jan 02 01:00:00 2020 | 124.00 + 8 | Wed Jan 01 20:00:00 2020 | 126.00 + 8 | Wed Jan 01 21:00:00 2020 | 126.00 + 8 | Wed Jan 01 22:00:00 2020 | 126.00 + 8 | Wed Jan 01 23:00:00 2020 | 126.00 + 8 | Thu Jan 02 00:00:00 2020 | 126.00 + 8 | Thu Jan 02 01:00:00 2020 | 126.00 + 9 | Wed Jan 01 20:00:00 2020 | 128.00 + 9 | Wed Jan 01 21:00:00 2020 | 128.00 + 9 | Wed Jan 01 22:00:00 2020 | 128.00 + 9 | Wed Jan 01 23:00:00 2020 | 128.00 + 9 | Thu Jan 02 00:00:00 2020 | 128.00 + 9 | Thu Jan 02 01:00:00 2020 | 128.00 +(60 rows) + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(11000, 12000) v; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 110.00 + 0 | Wed Jan 01 21:00:00 2020 | 110.00 + 0 | Wed Jan 01 22:00:00 2020 | 110.00 + 0 | Wed Jan 01 23:00:00 2020 | 110.00 + 0 | Thu Jan 02 00:00:00 2020 | 110.00 + 0 | Thu Jan 02 01:00:00 2020 | 82.50 + 1 | Wed Jan 01 20:00:00 2020 | 112.00 + 1 | Wed Jan 01 21:00:00 2020 | 112.00 + 1 | Wed Jan 01 22:00:00 2020 | 112.00 + 1 | Wed Jan 01 23:00:00 2020 | 112.00 + 1 | Thu Jan 02 00:00:00 2020 | 112.00 + 1 | Thu Jan 02 01:00:00 2020 | 79.71 + 2 | Wed Jan 01 20:00:00 2020 | 114.00 + 2 | Wed Jan 01 21:00:00 2020 | 114.00 + 2 | Wed Jan 01 22:00:00 2020 | 114.00 + 2 | Wed Jan 01 23:00:00 2020 | 114.00 + 2 | Thu Jan 02 00:00:00 2020 | 114.00 + 2 | Thu Jan 02 01:00:00 2020 | 81.18 + 3 | Wed Jan 01 20:00:00 2020 | 116.00 + 3 | Wed Jan 01 21:00:00 2020 | 116.00 + 3 | Wed Jan 01 22:00:00 2020 | 116.00 + 3 | Wed Jan 01 23:00:00 2020 | 116.00 + 3 | Thu Jan 02 00:00:00 2020 | 116.00 + 3 | Thu Jan 02 01:00:00 2020 | 82.65 + 4 | Wed Jan 01 20:00:00 2020 | 118.00 + 4 | Wed Jan 01 21:00:00 2020 | 118.00 + 4 | Wed Jan 01 22:00:00 2020 | 118.00 + 4 | Wed Jan 01 23:00:00 2020 | 118.00 + 4 | Thu Jan 02 00:00:00 2020 | 118.00 + 4 | Thu Jan 02 01:00:00 2020 | 84.12 + 5 | Wed Jan 01 20:00:00 2020 | 120.00 + 5 | Wed Jan 01 21:00:00 2020 | 120.00 + 5 | Wed Jan 01 22:00:00 2020 | 120.00 + 5 | Wed Jan 01 23:00:00 2020 | 120.00 + 5 | Thu Jan 02 00:00:00 2020 | 120.00 + 5 | Thu Jan 02 01:00:00 2020 | 85.59 + 6 | Wed Jan 01 20:00:00 2020 | 122.00 + 6 | Wed Jan 01 21:00:00 2020 | 122.00 + 6 | Wed Jan 01 22:00:00 2020 | 122.00 + 6 | Wed Jan 01 23:00:00 2020 | 122.00 + 6 | Thu Jan 02 00:00:00 2020 | 122.00 + 6 | Thu Jan 02 01:00:00 2020 | 87.06 + 7 | Wed Jan 01 20:00:00 2020 | 124.00 + 7 | Wed Jan 01 21:00:00 2020 | 124.00 + 7 | Wed Jan 01 22:00:00 2020 | 124.00 + 7 | Wed Jan 01 23:00:00 2020 | 124.00 + 7 | Thu Jan 02 00:00:00 2020 | 124.00 + 7 | Thu Jan 02 01:00:00 2020 | 88.53 + 8 | Wed Jan 01 20:00:00 2020 | 126.00 + 8 | Wed Jan 01 21:00:00 2020 | 126.00 + 8 | Wed Jan 01 22:00:00 2020 | 126.00 + 8 | Wed Jan 01 23:00:00 2020 | 126.00 + 8 | Thu Jan 02 00:00:00 2020 | 126.00 + 8 | Thu Jan 02 01:00:00 2020 | 90.00 + 9 | Wed Jan 01 20:00:00 2020 | 128.00 + 9 | Wed Jan 01 21:00:00 2020 | 128.00 + 9 | Wed Jan 01 22:00:00 2020 | 128.00 + 9 | Wed Jan 01 23:00:00 2020 | 128.00 + 9 | Thu Jan 02 00:00:00 2020 | 128.00 + 9 | Thu Jan 02 01:00:00 2020 | 91.47 +(60 rows) + +DELETE FROM events WHERE b < 100; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 110.00 + 0 | Wed Jan 01 21:00:00 2020 | 110.00 + 0 | Wed Jan 01 22:00:00 2020 | 110.00 + 0 | Wed Jan 01 23:00:00 2020 | 110.00 + 0 | Thu Jan 02 00:00:00 2020 | 110.00 + 0 | Thu Jan 02 01:00:00 2020 | 82.50 + 1 | Wed Jan 01 20:00:00 2020 | 112.00 + 1 | Wed Jan 01 21:00:00 2020 | 112.00 + 1 | Wed Jan 01 22:00:00 2020 | 112.00 + 1 | Wed Jan 01 23:00:00 2020 | 112.00 + 1 | Thu Jan 02 00:00:00 2020 | 112.00 + 1 | Thu Jan 02 01:00:00 2020 | 79.71 + 2 | Wed Jan 01 20:00:00 2020 | 114.00 + 2 | Wed Jan 01 21:00:00 2020 | 114.00 + 2 | Wed Jan 01 22:00:00 2020 | 114.00 + 2 | Wed Jan 01 23:00:00 2020 | 114.00 + 2 | Thu Jan 02 00:00:00 2020 | 114.00 + 2 | Thu Jan 02 01:00:00 2020 | 81.18 + 3 | Wed Jan 01 20:00:00 2020 | 116.00 + 3 | Wed Jan 01 21:00:00 2020 | 116.00 + 3 | Wed Jan 01 22:00:00 2020 | 116.00 + 3 | Wed Jan 01 23:00:00 2020 | 116.00 + 3 | Thu Jan 02 00:00:00 2020 | 116.00 + 3 | Thu Jan 02 01:00:00 2020 | 82.65 + 4 | Wed Jan 01 20:00:00 2020 | 118.00 + 4 | Wed Jan 01 21:00:00 2020 | 118.00 + 4 | Wed Jan 01 22:00:00 2020 | 118.00 + 4 | Wed Jan 01 23:00:00 2020 | 118.00 + 4 | Thu Jan 02 00:00:00 2020 | 118.00 + 4 | Thu Jan 02 01:00:00 2020 | 84.12 + 5 | Wed Jan 01 20:00:00 2020 | 120.00 + 5 | Wed Jan 01 21:00:00 2020 | 120.00 + 5 | Wed Jan 01 22:00:00 2020 | 120.00 + 5 | Wed Jan 01 23:00:00 2020 | 120.00 + 5 | Thu Jan 02 00:00:00 2020 | 120.00 + 5 | Thu Jan 02 01:00:00 2020 | 85.59 + 6 | Wed Jan 01 20:00:00 2020 | 122.00 + 6 | Wed Jan 01 21:00:00 2020 | 122.00 + 6 | Wed Jan 01 22:00:00 2020 | 122.00 + 6 | Wed Jan 01 23:00:00 2020 | 122.00 + 6 | Thu Jan 02 00:00:00 2020 | 122.00 + 6 | Thu Jan 02 01:00:00 2020 | 87.06 + 7 | Wed Jan 01 20:00:00 2020 | 124.00 + 7 | Wed Jan 01 21:00:00 2020 | 124.00 + 7 | Wed Jan 01 22:00:00 2020 | 124.00 + 7 | Wed Jan 01 23:00:00 2020 | 124.00 + 7 | Thu Jan 02 00:00:00 2020 | 124.00 + 7 | Thu Jan 02 01:00:00 2020 | 88.53 + 8 | Wed Jan 01 20:00:00 2020 | 126.00 + 8 | Wed Jan 01 21:00:00 2020 | 126.00 + 8 | Wed Jan 01 22:00:00 2020 | 126.00 + 8 | Wed Jan 01 23:00:00 2020 | 126.00 + 8 | Thu Jan 02 00:00:00 2020 | 126.00 + 8 | Thu Jan 02 01:00:00 2020 | 90.00 + 9 | Wed Jan 01 20:00:00 2020 | 128.00 + 9 | Wed Jan 01 21:00:00 2020 | 128.00 + 9 | Wed Jan 01 22:00:00 2020 | 128.00 + 9 | Wed Jan 01 23:00:00 2020 | 128.00 + 9 | Thu Jan 02 00:00:00 2020 | 128.00 + 9 | Thu Jan 02 01:00:00 2020 | 91.47 +(60 rows) + +UPDATE events SET b = b + b; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 110.00 + 0 | Wed Jan 01 21:00:00 2020 | 110.00 + 0 | Wed Jan 01 22:00:00 2020 | 110.00 + 0 | Wed Jan 01 23:00:00 2020 | 110.00 + 0 | Thu Jan 02 00:00:00 2020 | 110.00 + 0 | Thu Jan 02 01:00:00 2020 | 82.50 + 1 | Wed Jan 01 20:00:00 2020 | 112.00 + 1 | Wed Jan 01 21:00:00 2020 | 112.00 + 1 | Wed Jan 01 22:00:00 2020 | 112.00 + 1 | Wed Jan 01 23:00:00 2020 | 112.00 + 1 | Thu Jan 02 00:00:00 2020 | 112.00 + 1 | Thu Jan 02 01:00:00 2020 | 79.71 + 2 | Wed Jan 01 20:00:00 2020 | 114.00 + 2 | Wed Jan 01 21:00:00 2020 | 114.00 + 2 | Wed Jan 01 22:00:00 2020 | 114.00 + 2 | Wed Jan 01 23:00:00 2020 | 114.00 + 2 | Thu Jan 02 00:00:00 2020 | 114.00 + 2 | Thu Jan 02 01:00:00 2020 | 81.18 + 3 | Wed Jan 01 20:00:00 2020 | 116.00 + 3 | Wed Jan 01 21:00:00 2020 | 116.00 + 3 | Wed Jan 01 22:00:00 2020 | 116.00 + 3 | Wed Jan 01 23:00:00 2020 | 116.00 + 3 | Thu Jan 02 00:00:00 2020 | 116.00 + 3 | Thu Jan 02 01:00:00 2020 | 82.65 + 4 | Wed Jan 01 20:00:00 2020 | 118.00 + 4 | Wed Jan 01 21:00:00 2020 | 118.00 + 4 | Wed Jan 01 22:00:00 2020 | 118.00 + 4 | Wed Jan 01 23:00:00 2020 | 118.00 + 4 | Thu Jan 02 00:00:00 2020 | 118.00 + 4 | Thu Jan 02 01:00:00 2020 | 84.12 + 5 | Wed Jan 01 20:00:00 2020 | 120.00 + 5 | Wed Jan 01 21:00:00 2020 | 120.00 + 5 | Wed Jan 01 22:00:00 2020 | 120.00 + 5 | Wed Jan 01 23:00:00 2020 | 120.00 + 5 | Thu Jan 02 00:00:00 2020 | 120.00 + 5 | Thu Jan 02 01:00:00 2020 | 85.59 + 6 | Wed Jan 01 20:00:00 2020 | 122.00 + 6 | Wed Jan 01 21:00:00 2020 | 122.00 + 6 | Wed Jan 01 22:00:00 2020 | 122.00 + 6 | Wed Jan 01 23:00:00 2020 | 122.00 + 6 | Thu Jan 02 00:00:00 2020 | 122.00 + 6 | Thu Jan 02 01:00:00 2020 | 87.06 + 7 | Wed Jan 01 20:00:00 2020 | 124.00 + 7 | Wed Jan 01 21:00:00 2020 | 124.00 + 7 | Wed Jan 01 22:00:00 2020 | 124.00 + 7 | Wed Jan 01 23:00:00 2020 | 124.00 + 7 | Thu Jan 02 00:00:00 2020 | 124.00 + 7 | Thu Jan 02 01:00:00 2020 | 88.53 + 8 | Wed Jan 01 20:00:00 2020 | 126.00 + 8 | Wed Jan 01 21:00:00 2020 | 126.00 + 8 | Wed Jan 01 22:00:00 2020 | 126.00 + 8 | Wed Jan 01 23:00:00 2020 | 126.00 + 8 | Thu Jan 02 00:00:00 2020 | 126.00 + 8 | Thu Jan 02 01:00:00 2020 | 90.00 + 9 | Wed Jan 01 20:00:00 2020 | 128.00 + 9 | Wed Jan 01 21:00:00 2020 | 128.00 + 9 | Wed Jan 01 22:00:00 2020 | 128.00 + 9 | Wed Jan 01 23:00:00 2020 | 128.00 + 9 | Thu Jan 02 00:00:00 2020 | 128.00 + 9 | Thu Jan 02 01:00:00 2020 | 91.47 +(60 rows) + +REFRESH MATERIALIZED VIEW mv WITH NO DATA; +SELECT * FROM mv; + a | d_hour | avg_b +--------------------------------------------------------------------- +(0 rows) + +REFRESH MATERIALIZED VIEW mv; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 280.00 + 0 | Wed Jan 01 21:00:00 2020 | 280.00 + 0 | Wed Jan 01 22:00:00 2020 | 280.00 + 0 | Wed Jan 01 23:00:00 2020 | 280.00 + 0 | Thu Jan 02 00:00:00 2020 | 280.00 + 0 | Thu Jan 02 01:00:00 2020 | 280.00 + 1 | Wed Jan 01 20:00:00 2020 | 284.00 + 1 | Wed Jan 01 21:00:00 2020 | 284.00 + 1 | Wed Jan 01 22:00:00 2020 | 284.00 + 1 | Wed Jan 01 23:00:00 2020 | 284.00 + 1 | Thu Jan 02 00:00:00 2020 | 284.00 + 1 | Thu Jan 02 01:00:00 2020 | 284.00 + 2 | Wed Jan 01 20:00:00 2020 | 288.00 + 2 | Wed Jan 01 21:00:00 2020 | 288.00 + 2 | Wed Jan 01 22:00:00 2020 | 288.00 + 2 | Wed Jan 01 23:00:00 2020 | 288.00 + 2 | Thu Jan 02 00:00:00 2020 | 288.00 + 2 | Thu Jan 02 01:00:00 2020 | 288.00 + 3 | Wed Jan 01 20:00:00 2020 | 292.00 + 3 | Wed Jan 01 21:00:00 2020 | 292.00 + 3 | Wed Jan 01 22:00:00 2020 | 292.00 + 3 | Wed Jan 01 23:00:00 2020 | 292.00 + 3 | Thu Jan 02 00:00:00 2020 | 292.00 + 3 | Thu Jan 02 01:00:00 2020 | 292.00 + 4 | Wed Jan 01 20:00:00 2020 | 296.00 + 4 | Wed Jan 01 21:00:00 2020 | 296.00 + 4 | Wed Jan 01 22:00:00 2020 | 296.00 + 4 | Wed Jan 01 23:00:00 2020 | 296.00 + 4 | Thu Jan 02 00:00:00 2020 | 296.00 + 4 | Thu Jan 02 01:00:00 2020 | 296.00 + 5 | Wed Jan 01 20:00:00 2020 | 300.00 + 5 | Wed Jan 01 21:00:00 2020 | 300.00 + 5 | Wed Jan 01 22:00:00 2020 | 300.00 + 5 | Wed Jan 01 23:00:00 2020 | 300.00 + 5 | Thu Jan 02 00:00:00 2020 | 300.00 + 5 | Thu Jan 02 01:00:00 2020 | 300.00 + 6 | Wed Jan 01 20:00:00 2020 | 304.00 + 6 | Wed Jan 01 21:00:00 2020 | 304.00 + 6 | Wed Jan 01 22:00:00 2020 | 304.00 + 6 | Wed Jan 01 23:00:00 2020 | 304.00 + 6 | Thu Jan 02 00:00:00 2020 | 304.00 + 6 | Thu Jan 02 01:00:00 2020 | 304.00 + 7 | Wed Jan 01 20:00:00 2020 | 308.00 + 7 | Wed Jan 01 21:00:00 2020 | 308.00 + 7 | Wed Jan 01 22:00:00 2020 | 308.00 + 7 | Wed Jan 01 23:00:00 2020 | 308.00 + 7 | Thu Jan 02 00:00:00 2020 | 308.00 + 7 | Thu Jan 02 01:00:00 2020 | 308.00 + 8 | Wed Jan 01 20:00:00 2020 | 312.00 + 8 | Wed Jan 01 21:00:00 2020 | 312.00 + 8 | Wed Jan 01 22:00:00 2020 | 312.00 + 8 | Wed Jan 01 23:00:00 2020 | 312.00 + 8 | Thu Jan 02 00:00:00 2020 | 312.00 + 8 | Thu Jan 02 01:00:00 2020 | 312.00 + 9 | Wed Jan 01 20:00:00 2020 | 316.00 + 9 | Wed Jan 01 21:00:00 2020 | 316.00 + 9 | Wed Jan 01 22:00:00 2020 | 316.00 + 9 | Wed Jan 01 23:00:00 2020 | 316.00 + 9 | Thu Jan 02 00:00:00 2020 | 316.00 + 9 | Thu Jan 02 01:00:00 2020 | 316.00 +(60 rows) + +DROP MATERIALIZED VIEW mv; +NOTICE: drop cascades to trigger mv_INSERT on table events +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour WITH NO DATA; +SELECT * FROM mv; + a | d_hour | avg_b +--------------------------------------------------------------------- +(0 rows) + +DROP MATERIALIZED VIEW mv; +NOTICE: drop cascades to trigger mv_INSERT on table events +SELECT create_distributed_table('events', 'a'); +NOTICE: Copying data from local table... +NOTICE: copying the data has completed + create_distributed_table +--------------------------------------------------------------------- + +(1 row) + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + min(b) AS min_b, + max(b) AS max_b, + avg(b) AS avg_b, + min(c) AS min_c, + max(c) AS max_c, + avg(c) AS avg_c, + min(e) AS min_e, + max(e) AS max_e, + avg(e) AS avg_e +FROM events +WHERE b > 10 +GROUP BY a, d_hour; +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | min_b | max_b | avg_b | min_c | max_c | avg_c | min_e | max_e | avg_e +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 200.00 | 360.00 | 280.00 | 16.67 | 663.33 | 340.00 | 50.00 | 1990.00 | 1020.00 + 0 | Wed Jan 01 21:00:00 2020 | 200.00 | 360.00 | 280.00 | 683.33 | 1330.00 | 1006.67 | 2050.00 | 3990.00 | 3020.00 + 0 | Wed Jan 01 22:00:00 2020 | 200.00 | 360.00 | 280.00 | 1350.00 | 1996.67 | 1673.33 | 4050.00 | 5990.00 | 5020.00 + 0 | Wed Jan 01 23:00:00 2020 | 200.00 | 360.00 | 280.00 | 2016.67 | 2663.33 | 2340.00 | 6050.00 | 7990.00 | 7020.00 + 0 | Thu Jan 02 00:00:00 2020 | 200.00 | 360.00 | 280.00 | 2683.33 | 3330.00 | 3006.67 | 8050.00 | 9990.00 | 9020.00 + 0 | Thu Jan 02 01:00:00 2020 | 200.00 | 360.00 | 280.00 | 3350.00 | 3663.33 | 3506.67 | 10050.00 | 10990.00 | 10520.00 + 1 | Wed Jan 01 20:00:00 2020 | 204.00 | 364.00 | 284.00 | 17.00 | 663.67 | 340.33 | 51.00 | 1991.00 | 1021.00 + 1 | Wed Jan 01 21:00:00 2020 | 204.00 | 364.00 | 284.00 | 683.67 | 1330.33 | 1007.00 | 2051.00 | 3991.00 | 3021.00 + 1 | Wed Jan 01 22:00:00 2020 | 204.00 | 364.00 | 284.00 | 1350.33 | 1997.00 | 1673.67 | 4051.00 | 5991.00 | 5021.00 + 1 | Wed Jan 01 23:00:00 2020 | 204.00 | 364.00 | 284.00 | 2017.00 | 2663.67 | 2340.33 | 6051.00 | 7991.00 | 7021.00 + 1 | Thu Jan 02 00:00:00 2020 | 204.00 | 364.00 | 284.00 | 2683.67 | 3330.33 | 3007.00 | 8051.00 | 9991.00 | 9021.00 + 1 | Thu Jan 02 01:00:00 2020 | 204.00 | 364.00 | 284.00 | 3350.33 | 3663.67 | 3507.00 | 10051.00 | 10991.00 | 10521.00 + 2 | Wed Jan 01 20:00:00 2020 | 208.00 | 368.00 | 288.00 | 17.33 | 664.00 | 340.67 | 52.00 | 1992.00 | 1022.00 + 2 | Wed Jan 01 21:00:00 2020 | 208.00 | 368.00 | 288.00 | 684.00 | 1330.67 | 1007.33 | 2052.00 | 3992.00 | 3022.00 + 2 | Wed Jan 01 22:00:00 2020 | 208.00 | 368.00 | 288.00 | 1350.67 | 1997.33 | 1674.00 | 4052.00 | 5992.00 | 5022.00 + 2 | Wed Jan 01 23:00:00 2020 | 208.00 | 368.00 | 288.00 | 2017.33 | 2664.00 | 2340.67 | 6052.00 | 7992.00 | 7022.00 + 2 | Thu Jan 02 00:00:00 2020 | 208.00 | 368.00 | 288.00 | 2684.00 | 3330.67 | 3007.33 | 8052.00 | 9992.00 | 9022.00 + 2 | Thu Jan 02 01:00:00 2020 | 208.00 | 368.00 | 288.00 | 3350.67 | 3664.00 | 3507.33 | 10052.00 | 10992.00 | 10522.00 + 3 | Wed Jan 01 20:00:00 2020 | 212.00 | 372.00 | 292.00 | 17.67 | 664.33 | 341.00 | 53.00 | 1993.00 | 1023.00 + 3 | Wed Jan 01 21:00:00 2020 | 212.00 | 372.00 | 292.00 | 684.33 | 1331.00 | 1007.67 | 2053.00 | 3993.00 | 3023.00 + 3 | Wed Jan 01 22:00:00 2020 | 212.00 | 372.00 | 292.00 | 1351.00 | 1997.67 | 1674.33 | 4053.00 | 5993.00 | 5023.00 + 3 | Wed Jan 01 23:00:00 2020 | 212.00 | 372.00 | 292.00 | 2017.67 | 2664.33 | 2341.00 | 6053.00 | 7993.00 | 7023.00 + 3 | Thu Jan 02 00:00:00 2020 | 212.00 | 372.00 | 292.00 | 2684.33 | 3331.00 | 3007.67 | 8053.00 | 9993.00 | 9023.00 + 3 | Thu Jan 02 01:00:00 2020 | 212.00 | 372.00 | 292.00 | 3351.00 | 3664.33 | 3507.67 | 10053.00 | 10993.00 | 10523.00 + 4 | Wed Jan 01 20:00:00 2020 | 216.00 | 376.00 | 296.00 | 18.00 | 664.67 | 341.33 | 54.00 | 1994.00 | 1024.00 + 4 | Wed Jan 01 21:00:00 2020 | 216.00 | 376.00 | 296.00 | 684.67 | 1331.33 | 1008.00 | 2054.00 | 3994.00 | 3024.00 + 4 | Wed Jan 01 22:00:00 2020 | 216.00 | 376.00 | 296.00 | 1351.33 | 1998.00 | 1674.67 | 4054.00 | 5994.00 | 5024.00 + 4 | Wed Jan 01 23:00:00 2020 | 216.00 | 376.00 | 296.00 | 2018.00 | 2664.67 | 2341.33 | 6054.00 | 7994.00 | 7024.00 + 4 | Thu Jan 02 00:00:00 2020 | 216.00 | 376.00 | 296.00 | 2684.67 | 3331.33 | 3008.00 | 8054.00 | 9994.00 | 9024.00 + 4 | Thu Jan 02 01:00:00 2020 | 216.00 | 376.00 | 296.00 | 3351.33 | 3664.67 | 3508.00 | 10054.00 | 10994.00 | 10524.00 + 5 | Wed Jan 01 20:00:00 2020 | 220.00 | 380.00 | 300.00 | 18.33 | 665.00 | 341.67 | 55.00 | 1995.00 | 1025.00 + 5 | Wed Jan 01 21:00:00 2020 | 220.00 | 380.00 | 300.00 | 685.00 | 1331.67 | 1008.33 | 2055.00 | 3995.00 | 3025.00 + 5 | Wed Jan 01 22:00:00 2020 | 220.00 | 380.00 | 300.00 | 1351.67 | 1998.33 | 1675.00 | 4055.00 | 5995.00 | 5025.00 + 5 | Wed Jan 01 23:00:00 2020 | 220.00 | 380.00 | 300.00 | 2018.33 | 2665.00 | 2341.67 | 6055.00 | 7995.00 | 7025.00 + 5 | Thu Jan 02 00:00:00 2020 | 220.00 | 380.00 | 300.00 | 2685.00 | 3331.67 | 3008.33 | 8055.00 | 9995.00 | 9025.00 + 5 | Thu Jan 02 01:00:00 2020 | 220.00 | 380.00 | 300.00 | 3351.67 | 3665.00 | 3508.33 | 10055.00 | 10995.00 | 10525.00 + 6 | Wed Jan 01 20:00:00 2020 | 224.00 | 384.00 | 304.00 | 18.67 | 665.33 | 342.00 | 56.00 | 1996.00 | 1026.00 + 6 | Wed Jan 01 21:00:00 2020 | 224.00 | 384.00 | 304.00 | 685.33 | 1332.00 | 1008.67 | 2056.00 | 3996.00 | 3026.00 + 6 | Wed Jan 01 22:00:00 2020 | 224.00 | 384.00 | 304.00 | 1352.00 | 1998.67 | 1675.33 | 4056.00 | 5996.00 | 5026.00 + 6 | Wed Jan 01 23:00:00 2020 | 224.00 | 384.00 | 304.00 | 2018.67 | 2665.33 | 2342.00 | 6056.00 | 7996.00 | 7026.00 + 6 | Thu Jan 02 00:00:00 2020 | 224.00 | 384.00 | 304.00 | 2685.33 | 3332.00 | 3008.67 | 8056.00 | 9996.00 | 9026.00 + 6 | Thu Jan 02 01:00:00 2020 | 224.00 | 384.00 | 304.00 | 3352.00 | 3665.33 | 3508.67 | 10056.00 | 10996.00 | 10526.00 + 7 | Wed Jan 01 20:00:00 2020 | 228.00 | 388.00 | 308.00 | 19.00 | 665.67 | 342.33 | 57.00 | 1997.00 | 1027.00 + 7 | Wed Jan 01 21:00:00 2020 | 228.00 | 388.00 | 308.00 | 685.67 | 1332.33 | 1009.00 | 2057.00 | 3997.00 | 3027.00 + 7 | Wed Jan 01 22:00:00 2020 | 228.00 | 388.00 | 308.00 | 1352.33 | 1999.00 | 1675.67 | 4057.00 | 5997.00 | 5027.00 + 7 | Wed Jan 01 23:00:00 2020 | 228.00 | 388.00 | 308.00 | 2019.00 | 2665.67 | 2342.33 | 6057.00 | 7997.00 | 7027.00 + 7 | Thu Jan 02 00:00:00 2020 | 228.00 | 388.00 | 308.00 | 2685.67 | 3332.33 | 3009.00 | 8057.00 | 9997.00 | 9027.00 + 7 | Thu Jan 02 01:00:00 2020 | 228.00 | 388.00 | 308.00 | 3352.33 | 3665.67 | 3509.00 | 10057.00 | 10997.00 | 10527.00 + 8 | Wed Jan 01 20:00:00 2020 | 232.00 | 392.00 | 312.00 | 19.33 | 666.00 | 342.67 | 58.00 | 1998.00 | 1028.00 + 8 | Wed Jan 01 21:00:00 2020 | 232.00 | 392.00 | 312.00 | 686.00 | 1332.67 | 1009.33 | 2058.00 | 3998.00 | 3028.00 + 8 | Wed Jan 01 22:00:00 2020 | 232.00 | 392.00 | 312.00 | 1352.67 | 1999.33 | 1676.00 | 4058.00 | 5998.00 | 5028.00 + 8 | Wed Jan 01 23:00:00 2020 | 232.00 | 392.00 | 312.00 | 2019.33 | 2666.00 | 2342.67 | 6058.00 | 7998.00 | 7028.00 + 8 | Thu Jan 02 00:00:00 2020 | 232.00 | 392.00 | 312.00 | 2686.00 | 3332.67 | 3009.33 | 8058.00 | 9998.00 | 9028.00 + 8 | Thu Jan 02 01:00:00 2020 | 232.00 | 392.00 | 312.00 | 3352.67 | 3666.00 | 3509.33 | 10058.00 | 10998.00 | 10528.00 + 9 | Wed Jan 01 20:00:00 2020 | 236.00 | 396.00 | 316.00 | 19.67 | 666.33 | 343.00 | 59.00 | 1999.00 | 1029.00 + 9 | Wed Jan 01 21:00:00 2020 | 236.00 | 396.00 | 316.00 | 686.33 | 1333.00 | 1009.67 | 2059.00 | 3999.00 | 3029.00 + 9 | Wed Jan 01 22:00:00 2020 | 236.00 | 396.00 | 316.00 | 1353.00 | 1999.67 | 1676.33 | 4059.00 | 5999.00 | 5029.00 + 9 | Wed Jan 01 23:00:00 2020 | 236.00 | 396.00 | 316.00 | 2019.67 | 2666.33 | 2343.00 | 6059.00 | 7999.00 | 7029.00 + 9 | Thu Jan 02 00:00:00 2020 | 236.00 | 396.00 | 316.00 | 2686.33 | 3333.00 | 3009.67 | 8059.00 | 9999.00 | 9029.00 + 9 | Thu Jan 02 01:00:00 2020 | 236.00 | 396.00 | 316.00 | 3353.00 | 3666.33 | 3509.67 | 10059.00 | 10999.00 | 10529.00 +(60 rows) + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(12000, 13000) v; +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | min_b | max_b | avg_b | min_c | max_c | avg_c | min_e | max_e | avg_e +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 200.00 | 360.00 | 280.00 | 16.67 | 663.33 | 340.00 | 50.00 | 1990.00 | 1020.00 + 0 | Wed Jan 01 21:00:00 2020 | 200.00 | 360.00 | 280.00 | 683.33 | 1330.00 | 1006.67 | 2050.00 | 3990.00 | 3020.00 + 0 | Wed Jan 01 22:00:00 2020 | 200.00 | 360.00 | 280.00 | 1350.00 | 1996.67 | 1673.33 | 4050.00 | 5990.00 | 5020.00 + 0 | Wed Jan 01 23:00:00 2020 | 200.00 | 360.00 | 280.00 | 2016.67 | 2663.33 | 2340.00 | 6050.00 | 7990.00 | 7020.00 + 0 | Thu Jan 02 00:00:00 2020 | 200.00 | 360.00 | 280.00 | 2683.33 | 3330.00 | 3006.67 | 8050.00 | 9990.00 | 9020.00 + 0 | Thu Jan 02 01:00:00 2020 | 200.00 | 360.00 | 280.00 | 3350.00 | 3663.33 | 3506.67 | 10050.00 | 10990.00 | 10520.00 + 0 | Thu Jan 02 02:00:00 2020 | 20.00 | 90.00 | 55.00 | 4006.67 | 4330.00 | 4168.33 | 12020.00 | 12990.00 | 12505.00 + 1 | Wed Jan 01 20:00:00 2020 | 204.00 | 364.00 | 284.00 | 17.00 | 663.67 | 340.33 | 51.00 | 1991.00 | 1021.00 + 1 | Wed Jan 01 21:00:00 2020 | 204.00 | 364.00 | 284.00 | 683.67 | 1330.33 | 1007.00 | 2051.00 | 3991.00 | 3021.00 + 1 | Wed Jan 01 22:00:00 2020 | 204.00 | 364.00 | 284.00 | 1350.33 | 1997.00 | 1673.67 | 4051.00 | 5991.00 | 5021.00 + 1 | Wed Jan 01 23:00:00 2020 | 204.00 | 364.00 | 284.00 | 2017.00 | 2663.67 | 2340.33 | 6051.00 | 7991.00 | 7021.00 + 1 | Thu Jan 02 00:00:00 2020 | 204.00 | 364.00 | 284.00 | 2683.67 | 3330.33 | 3007.00 | 8051.00 | 9991.00 | 9021.00 + 1 | Thu Jan 02 01:00:00 2020 | 204.00 | 364.00 | 284.00 | 3350.33 | 3663.67 | 3507.00 | 10051.00 | 10991.00 | 10521.00 + 1 | Thu Jan 02 02:00:00 2020 | 11.00 | 91.00 | 51.00 | 4003.67 | 4330.33 | 4167.00 | 12011.00 | 12991.00 | 12501.00 + 2 | Wed Jan 01 20:00:00 2020 | 208.00 | 368.00 | 288.00 | 17.33 | 664.00 | 340.67 | 52.00 | 1992.00 | 1022.00 + 2 | Wed Jan 01 21:00:00 2020 | 208.00 | 368.00 | 288.00 | 684.00 | 1330.67 | 1007.33 | 2052.00 | 3992.00 | 3022.00 + 2 | Wed Jan 01 22:00:00 2020 | 208.00 | 368.00 | 288.00 | 1350.67 | 1997.33 | 1674.00 | 4052.00 | 5992.00 | 5022.00 + 2 | Wed Jan 01 23:00:00 2020 | 208.00 | 368.00 | 288.00 | 2017.33 | 2664.00 | 2340.67 | 6052.00 | 7992.00 | 7022.00 + 2 | Thu Jan 02 00:00:00 2020 | 208.00 | 368.00 | 288.00 | 2684.00 | 3330.67 | 3007.33 | 8052.00 | 9992.00 | 9022.00 + 2 | Thu Jan 02 01:00:00 2020 | 208.00 | 368.00 | 288.00 | 3350.67 | 3664.00 | 3507.33 | 10052.00 | 10992.00 | 10522.00 + 2 | Thu Jan 02 02:00:00 2020 | 12.00 | 92.00 | 52.00 | 4004.00 | 4330.67 | 4167.33 | 12012.00 | 12992.00 | 12502.00 + 3 | Wed Jan 01 20:00:00 2020 | 212.00 | 372.00 | 292.00 | 17.67 | 664.33 | 341.00 | 53.00 | 1993.00 | 1023.00 + 3 | Wed Jan 01 21:00:00 2020 | 212.00 | 372.00 | 292.00 | 684.33 | 1331.00 | 1007.67 | 2053.00 | 3993.00 | 3023.00 + 3 | Wed Jan 01 22:00:00 2020 | 212.00 | 372.00 | 292.00 | 1351.00 | 1997.67 | 1674.33 | 4053.00 | 5993.00 | 5023.00 + 3 | Wed Jan 01 23:00:00 2020 | 212.00 | 372.00 | 292.00 | 2017.67 | 2664.33 | 2341.00 | 6053.00 | 7993.00 | 7023.00 + 3 | Thu Jan 02 00:00:00 2020 | 212.00 | 372.00 | 292.00 | 2684.33 | 3331.00 | 3007.67 | 8053.00 | 9993.00 | 9023.00 + 3 | Thu Jan 02 01:00:00 2020 | 212.00 | 372.00 | 292.00 | 3351.00 | 3664.33 | 3507.67 | 10053.00 | 10993.00 | 10523.00 + 3 | Thu Jan 02 02:00:00 2020 | 13.00 | 93.00 | 53.00 | 4004.33 | 4331.00 | 4167.67 | 12013.00 | 12993.00 | 12503.00 + 4 | Wed Jan 01 20:00:00 2020 | 216.00 | 376.00 | 296.00 | 18.00 | 664.67 | 341.33 | 54.00 | 1994.00 | 1024.00 + 4 | Wed Jan 01 21:00:00 2020 | 216.00 | 376.00 | 296.00 | 684.67 | 1331.33 | 1008.00 | 2054.00 | 3994.00 | 3024.00 + 4 | Wed Jan 01 22:00:00 2020 | 216.00 | 376.00 | 296.00 | 1351.33 | 1998.00 | 1674.67 | 4054.00 | 5994.00 | 5024.00 + 4 | Wed Jan 01 23:00:00 2020 | 216.00 | 376.00 | 296.00 | 2018.00 | 2664.67 | 2341.33 | 6054.00 | 7994.00 | 7024.00 + 4 | Thu Jan 02 00:00:00 2020 | 216.00 | 376.00 | 296.00 | 2684.67 | 3331.33 | 3008.00 | 8054.00 | 9994.00 | 9024.00 + 4 | Thu Jan 02 01:00:00 2020 | 216.00 | 376.00 | 296.00 | 3351.33 | 3664.67 | 3508.00 | 10054.00 | 10994.00 | 10524.00 + 4 | Thu Jan 02 02:00:00 2020 | 14.00 | 94.00 | 54.00 | 4004.67 | 4331.33 | 4168.00 | 12014.00 | 12994.00 | 12504.00 + 5 | Wed Jan 01 20:00:00 2020 | 220.00 | 380.00 | 300.00 | 18.33 | 665.00 | 341.67 | 55.00 | 1995.00 | 1025.00 + 5 | Wed Jan 01 21:00:00 2020 | 220.00 | 380.00 | 300.00 | 685.00 | 1331.67 | 1008.33 | 2055.00 | 3995.00 | 3025.00 + 5 | Wed Jan 01 22:00:00 2020 | 220.00 | 380.00 | 300.00 | 1351.67 | 1998.33 | 1675.00 | 4055.00 | 5995.00 | 5025.00 + 5 | Wed Jan 01 23:00:00 2020 | 220.00 | 380.00 | 300.00 | 2018.33 | 2665.00 | 2341.67 | 6055.00 | 7995.00 | 7025.00 + 5 | Thu Jan 02 00:00:00 2020 | 220.00 | 380.00 | 300.00 | 2685.00 | 3331.67 | 3008.33 | 8055.00 | 9995.00 | 9025.00 + 5 | Thu Jan 02 01:00:00 2020 | 220.00 | 380.00 | 300.00 | 3351.67 | 3665.00 | 3508.33 | 10055.00 | 10995.00 | 10525.00 + 5 | Thu Jan 02 02:00:00 2020 | 15.00 | 95.00 | 55.00 | 4005.00 | 4331.67 | 4168.33 | 12015.00 | 12995.00 | 12505.00 + 6 | Wed Jan 01 20:00:00 2020 | 224.00 | 384.00 | 304.00 | 18.67 | 665.33 | 342.00 | 56.00 | 1996.00 | 1026.00 + 6 | Wed Jan 01 21:00:00 2020 | 224.00 | 384.00 | 304.00 | 685.33 | 1332.00 | 1008.67 | 2056.00 | 3996.00 | 3026.00 + 6 | Wed Jan 01 22:00:00 2020 | 224.00 | 384.00 | 304.00 | 1352.00 | 1998.67 | 1675.33 | 4056.00 | 5996.00 | 5026.00 + 6 | Wed Jan 01 23:00:00 2020 | 224.00 | 384.00 | 304.00 | 2018.67 | 2665.33 | 2342.00 | 6056.00 | 7996.00 | 7026.00 + 6 | Thu Jan 02 00:00:00 2020 | 224.00 | 384.00 | 304.00 | 2685.33 | 3332.00 | 3008.67 | 8056.00 | 9996.00 | 9026.00 + 6 | Thu Jan 02 01:00:00 2020 | 224.00 | 384.00 | 304.00 | 3352.00 | 3665.33 | 3508.67 | 10056.00 | 10996.00 | 10526.00 + 6 | Thu Jan 02 02:00:00 2020 | 16.00 | 96.00 | 56.00 | 4005.33 | 4332.00 | 4168.67 | 12016.00 | 12996.00 | 12506.00 + 7 | Wed Jan 01 20:00:00 2020 | 228.00 | 388.00 | 308.00 | 19.00 | 665.67 | 342.33 | 57.00 | 1997.00 | 1027.00 + 7 | Wed Jan 01 21:00:00 2020 | 228.00 | 388.00 | 308.00 | 685.67 | 1332.33 | 1009.00 | 2057.00 | 3997.00 | 3027.00 + 7 | Wed Jan 01 22:00:00 2020 | 228.00 | 388.00 | 308.00 | 1352.33 | 1999.00 | 1675.67 | 4057.00 | 5997.00 | 5027.00 + 7 | Wed Jan 01 23:00:00 2020 | 228.00 | 388.00 | 308.00 | 2019.00 | 2665.67 | 2342.33 | 6057.00 | 7997.00 | 7027.00 + 7 | Thu Jan 02 00:00:00 2020 | 228.00 | 388.00 | 308.00 | 2685.67 | 3332.33 | 3009.00 | 8057.00 | 9997.00 | 9027.00 + 7 | Thu Jan 02 01:00:00 2020 | 228.00 | 388.00 | 308.00 | 3352.33 | 3665.67 | 3509.00 | 10057.00 | 10997.00 | 10527.00 + 7 | Thu Jan 02 02:00:00 2020 | 17.00 | 97.00 | 57.00 | 4005.67 | 4332.33 | 4169.00 | 12017.00 | 12997.00 | 12507.00 + 8 | Wed Jan 01 20:00:00 2020 | 232.00 | 392.00 | 312.00 | 19.33 | 666.00 | 342.67 | 58.00 | 1998.00 | 1028.00 + 8 | Wed Jan 01 21:00:00 2020 | 232.00 | 392.00 | 312.00 | 686.00 | 1332.67 | 1009.33 | 2058.00 | 3998.00 | 3028.00 + 8 | Wed Jan 01 22:00:00 2020 | 232.00 | 392.00 | 312.00 | 1352.67 | 1999.33 | 1676.00 | 4058.00 | 5998.00 | 5028.00 + 8 | Wed Jan 01 23:00:00 2020 | 232.00 | 392.00 | 312.00 | 2019.33 | 2666.00 | 2342.67 | 6058.00 | 7998.00 | 7028.00 + 8 | Thu Jan 02 00:00:00 2020 | 232.00 | 392.00 | 312.00 | 2686.00 | 3332.67 | 3009.33 | 8058.00 | 9998.00 | 9028.00 + 8 | Thu Jan 02 01:00:00 2020 | 232.00 | 392.00 | 312.00 | 3352.67 | 3666.00 | 3509.33 | 10058.00 | 10998.00 | 10528.00 + 8 | Thu Jan 02 02:00:00 2020 | 18.00 | 98.00 | 58.00 | 4006.00 | 4332.67 | 4169.33 | 12018.00 | 12998.00 | 12508.00 + 9 | Wed Jan 01 20:00:00 2020 | 236.00 | 396.00 | 316.00 | 19.67 | 666.33 | 343.00 | 59.00 | 1999.00 | 1029.00 + 9 | Wed Jan 01 21:00:00 2020 | 236.00 | 396.00 | 316.00 | 686.33 | 1333.00 | 1009.67 | 2059.00 | 3999.00 | 3029.00 + 9 | Wed Jan 01 22:00:00 2020 | 236.00 | 396.00 | 316.00 | 1353.00 | 1999.67 | 1676.33 | 4059.00 | 5999.00 | 5029.00 + 9 | Wed Jan 01 23:00:00 2020 | 236.00 | 396.00 | 316.00 | 2019.67 | 2666.33 | 2343.00 | 6059.00 | 7999.00 | 7029.00 + 9 | Thu Jan 02 00:00:00 2020 | 236.00 | 396.00 | 316.00 | 2686.33 | 3333.00 | 3009.67 | 8059.00 | 9999.00 | 9029.00 + 9 | Thu Jan 02 01:00:00 2020 | 236.00 | 396.00 | 316.00 | 3353.00 | 3666.33 | 3509.67 | 10059.00 | 10999.00 | 10529.00 + 9 | Thu Jan 02 02:00:00 2020 | 19.00 | 99.00 | 59.00 | 4006.33 | 4333.00 | 4169.67 | 12019.00 | 12999.00 | 12509.00 +(70 rows) + +DELETE FROM events WHERE b < 100; +ERROR: MATERIALIZED VIEW 'mv' on table 'events' does not support UPDATE/DELETE +DROP VIEW mv; +ERROR: DROP VIEW not supported for mv +DROP MATERIALIZED VIEW mv; +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 280.00 + 0 | Wed Jan 01 21:00:00 2020 | 280.00 + 0 | Wed Jan 01 22:00:00 2020 | 280.00 + 0 | Wed Jan 01 23:00:00 2020 | 280.00 + 0 | Thu Jan 02 00:00:00 2020 | 280.00 + 0 | Thu Jan 02 01:00:00 2020 | 280.00 + 0 | Thu Jan 02 02:00:00 2020 | 55.00 + 1 | Wed Jan 01 20:00:00 2020 | 284.00 + 1 | Wed Jan 01 21:00:00 2020 | 284.00 + 1 | Wed Jan 01 22:00:00 2020 | 284.00 + 1 | Wed Jan 01 23:00:00 2020 | 284.00 + 1 | Thu Jan 02 00:00:00 2020 | 284.00 + 1 | Thu Jan 02 01:00:00 2020 | 284.00 + 1 | Thu Jan 02 02:00:00 2020 | 51.00 + 2 | Wed Jan 01 20:00:00 2020 | 288.00 + 2 | Wed Jan 01 21:00:00 2020 | 288.00 + 2 | Wed Jan 01 22:00:00 2020 | 288.00 + 2 | Wed Jan 01 23:00:00 2020 | 288.00 + 2 | Thu Jan 02 00:00:00 2020 | 288.00 + 2 | Thu Jan 02 01:00:00 2020 | 288.00 + 2 | Thu Jan 02 02:00:00 2020 | 52.00 + 3 | Wed Jan 01 20:00:00 2020 | 292.00 + 3 | Wed Jan 01 21:00:00 2020 | 292.00 + 3 | Wed Jan 01 22:00:00 2020 | 292.00 + 3 | Wed Jan 01 23:00:00 2020 | 292.00 + 3 | Thu Jan 02 00:00:00 2020 | 292.00 + 3 | Thu Jan 02 01:00:00 2020 | 292.00 + 3 | Thu Jan 02 02:00:00 2020 | 53.00 + 4 | Wed Jan 01 20:00:00 2020 | 296.00 + 4 | Wed Jan 01 21:00:00 2020 | 296.00 + 4 | Wed Jan 01 22:00:00 2020 | 296.00 + 4 | Wed Jan 01 23:00:00 2020 | 296.00 + 4 | Thu Jan 02 00:00:00 2020 | 296.00 + 4 | Thu Jan 02 01:00:00 2020 | 296.00 + 4 | Thu Jan 02 02:00:00 2020 | 54.00 + 5 | Wed Jan 01 20:00:00 2020 | 300.00 + 5 | Wed Jan 01 21:00:00 2020 | 300.00 + 5 | Wed Jan 01 22:00:00 2020 | 300.00 + 5 | Wed Jan 01 23:00:00 2020 | 300.00 + 5 | Thu Jan 02 00:00:00 2020 | 300.00 + 5 | Thu Jan 02 01:00:00 2020 | 300.00 + 5 | Thu Jan 02 02:00:00 2020 | 55.00 + 6 | Wed Jan 01 20:00:00 2020 | 304.00 + 6 | Wed Jan 01 21:00:00 2020 | 304.00 + 6 | Wed Jan 01 22:00:00 2020 | 304.00 + 6 | Wed Jan 01 23:00:00 2020 | 304.00 + 6 | Thu Jan 02 00:00:00 2020 | 304.00 + 6 | Thu Jan 02 01:00:00 2020 | 304.00 + 6 | Thu Jan 02 02:00:00 2020 | 56.00 + 7 | Wed Jan 01 20:00:00 2020 | 308.00 + 7 | Wed Jan 01 21:00:00 2020 | 308.00 + 7 | Wed Jan 01 22:00:00 2020 | 308.00 + 7 | Wed Jan 01 23:00:00 2020 | 308.00 + 7 | Thu Jan 02 00:00:00 2020 | 308.00 + 7 | Thu Jan 02 01:00:00 2020 | 308.00 + 7 | Thu Jan 02 02:00:00 2020 | 57.00 + 8 | Wed Jan 01 20:00:00 2020 | 312.00 + 8 | Wed Jan 01 21:00:00 2020 | 312.00 + 8 | Wed Jan 01 22:00:00 2020 | 312.00 + 8 | Wed Jan 01 23:00:00 2020 | 312.00 + 8 | Thu Jan 02 00:00:00 2020 | 312.00 + 8 | Thu Jan 02 01:00:00 2020 | 312.00 + 8 | Thu Jan 02 02:00:00 2020 | 58.00 + 9 | Wed Jan 01 20:00:00 2020 | 316.00 + 9 | Wed Jan 01 21:00:00 2020 | 316.00 + 9 | Wed Jan 01 22:00:00 2020 | 316.00 + 9 | Wed Jan 01 23:00:00 2020 | 316.00 + 9 | Thu Jan 02 00:00:00 2020 | 316.00 + 9 | Thu Jan 02 01:00:00 2020 | 316.00 + 9 | Thu Jan 02 02:00:00 2020 | 59.00 +(70 rows) + +DELETE FROM events WHERE b < 20; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 280.00 + 0 | Wed Jan 01 21:00:00 2020 | 280.00 + 0 | Wed Jan 01 22:00:00 2020 | 280.00 + 0 | Wed Jan 01 23:00:00 2020 | 280.00 + 0 | Thu Jan 02 00:00:00 2020 | 280.00 + 0 | Thu Jan 02 01:00:00 2020 | 280.00 + 0 | Thu Jan 02 02:00:00 2020 | 55.00 + 1 | Wed Jan 01 20:00:00 2020 | 284.00 + 1 | Wed Jan 01 21:00:00 2020 | 284.00 + 1 | Wed Jan 01 22:00:00 2020 | 284.00 + 1 | Wed Jan 01 23:00:00 2020 | 284.00 + 1 | Thu Jan 02 00:00:00 2020 | 284.00 + 1 | Thu Jan 02 01:00:00 2020 | 284.00 + 1 | Thu Jan 02 02:00:00 2020 | 56.00 + 2 | Wed Jan 01 20:00:00 2020 | 288.00 + 2 | Wed Jan 01 21:00:00 2020 | 288.00 + 2 | Wed Jan 01 22:00:00 2020 | 288.00 + 2 | Wed Jan 01 23:00:00 2020 | 288.00 + 2 | Thu Jan 02 00:00:00 2020 | 288.00 + 2 | Thu Jan 02 01:00:00 2020 | 288.00 + 2 | Thu Jan 02 02:00:00 2020 | 57.00 + 3 | Wed Jan 01 20:00:00 2020 | 292.00 + 3 | Wed Jan 01 21:00:00 2020 | 292.00 + 3 | Wed Jan 01 22:00:00 2020 | 292.00 + 3 | Wed Jan 01 23:00:00 2020 | 292.00 + 3 | Thu Jan 02 00:00:00 2020 | 292.00 + 3 | Thu Jan 02 01:00:00 2020 | 292.00 + 3 | Thu Jan 02 02:00:00 2020 | 58.00 + 4 | Wed Jan 01 20:00:00 2020 | 296.00 + 4 | Wed Jan 01 21:00:00 2020 | 296.00 + 4 | Wed Jan 01 22:00:00 2020 | 296.00 + 4 | Wed Jan 01 23:00:00 2020 | 296.00 + 4 | Thu Jan 02 00:00:00 2020 | 296.00 + 4 | Thu Jan 02 01:00:00 2020 | 296.00 + 4 | Thu Jan 02 02:00:00 2020 | 59.00 + 5 | Wed Jan 01 20:00:00 2020 | 300.00 + 5 | Wed Jan 01 21:00:00 2020 | 300.00 + 5 | Wed Jan 01 22:00:00 2020 | 300.00 + 5 | Wed Jan 01 23:00:00 2020 | 300.00 + 5 | Thu Jan 02 00:00:00 2020 | 300.00 + 5 | Thu Jan 02 01:00:00 2020 | 300.00 + 5 | Thu Jan 02 02:00:00 2020 | 60.00 + 6 | Wed Jan 01 20:00:00 2020 | 304.00 + 6 | Wed Jan 01 21:00:00 2020 | 304.00 + 6 | Wed Jan 01 22:00:00 2020 | 304.00 + 6 | Wed Jan 01 23:00:00 2020 | 304.00 + 6 | Thu Jan 02 00:00:00 2020 | 304.00 + 6 | Thu Jan 02 01:00:00 2020 | 304.00 + 6 | Thu Jan 02 02:00:00 2020 | 61.00 + 7 | Wed Jan 01 20:00:00 2020 | 308.00 + 7 | Wed Jan 01 21:00:00 2020 | 308.00 + 7 | Wed Jan 01 22:00:00 2020 | 308.00 + 7 | Wed Jan 01 23:00:00 2020 | 308.00 + 7 | Thu Jan 02 00:00:00 2020 | 308.00 + 7 | Thu Jan 02 01:00:00 2020 | 308.00 + 7 | Thu Jan 02 02:00:00 2020 | 62.00 + 8 | Wed Jan 01 20:00:00 2020 | 312.00 + 8 | Wed Jan 01 21:00:00 2020 | 312.00 + 8 | Wed Jan 01 22:00:00 2020 | 312.00 + 8 | Wed Jan 01 23:00:00 2020 | 312.00 + 8 | Thu Jan 02 00:00:00 2020 | 312.00 + 8 | Thu Jan 02 01:00:00 2020 | 312.00 + 8 | Thu Jan 02 02:00:00 2020 | 63.00 + 9 | Wed Jan 01 20:00:00 2020 | 316.00 + 9 | Wed Jan 01 21:00:00 2020 | 316.00 + 9 | Wed Jan 01 22:00:00 2020 | 316.00 + 9 | Wed Jan 01 23:00:00 2020 | 316.00 + 9 | Thu Jan 02 00:00:00 2020 | 316.00 + 9 | Thu Jan 02 01:00:00 2020 | 316.00 + 9 | Thu Jan 02 02:00:00 2020 | 64.00 +(70 rows) + +UPDATE events SET b = b + b; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 560.00 + 0 | Wed Jan 01 21:00:00 2020 | 560.00 + 0 | Wed Jan 01 22:00:00 2020 | 560.00 + 0 | Wed Jan 01 23:00:00 2020 | 560.00 + 0 | Thu Jan 02 00:00:00 2020 | 560.00 + 0 | Thu Jan 02 01:00:00 2020 | 560.00 + 0 | Thu Jan 02 02:00:00 2020 | 110.00 + 1 | Wed Jan 01 20:00:00 2020 | 568.00 + 1 | Wed Jan 01 21:00:00 2020 | 568.00 + 1 | Wed Jan 01 22:00:00 2020 | 568.00 + 1 | Wed Jan 01 23:00:00 2020 | 568.00 + 1 | Thu Jan 02 00:00:00 2020 | 568.00 + 1 | Thu Jan 02 01:00:00 2020 | 568.00 + 1 | Thu Jan 02 02:00:00 2020 | 112.00 + 2 | Wed Jan 01 20:00:00 2020 | 576.00 + 2 | Wed Jan 01 21:00:00 2020 | 576.00 + 2 | Wed Jan 01 22:00:00 2020 | 576.00 + 2 | Wed Jan 01 23:00:00 2020 | 576.00 + 2 | Thu Jan 02 00:00:00 2020 | 576.00 + 2 | Thu Jan 02 01:00:00 2020 | 576.00 + 2 | Thu Jan 02 02:00:00 2020 | 114.00 + 3 | Wed Jan 01 20:00:00 2020 | 584.00 + 3 | Wed Jan 01 21:00:00 2020 | 584.00 + 3 | Wed Jan 01 22:00:00 2020 | 584.00 + 3 | Wed Jan 01 23:00:00 2020 | 584.00 + 3 | Thu Jan 02 00:00:00 2020 | 584.00 + 3 | Thu Jan 02 01:00:00 2020 | 584.00 + 3 | Thu Jan 02 02:00:00 2020 | 116.00 + 4 | Wed Jan 01 20:00:00 2020 | 592.00 + 4 | Wed Jan 01 21:00:00 2020 | 592.00 + 4 | Wed Jan 01 22:00:00 2020 | 592.00 + 4 | Wed Jan 01 23:00:00 2020 | 592.00 + 4 | Thu Jan 02 00:00:00 2020 | 592.00 + 4 | Thu Jan 02 01:00:00 2020 | 592.00 + 4 | Thu Jan 02 02:00:00 2020 | 118.00 + 5 | Wed Jan 01 20:00:00 2020 | 600.00 + 5 | Wed Jan 01 21:00:00 2020 | 600.00 + 5 | Wed Jan 01 22:00:00 2020 | 600.00 + 5 | Wed Jan 01 23:00:00 2020 | 600.00 + 5 | Thu Jan 02 00:00:00 2020 | 600.00 + 5 | Thu Jan 02 01:00:00 2020 | 600.00 + 5 | Thu Jan 02 02:00:00 2020 | 120.00 + 6 | Wed Jan 01 20:00:00 2020 | 608.00 + 6 | Wed Jan 01 21:00:00 2020 | 608.00 + 6 | Wed Jan 01 22:00:00 2020 | 608.00 + 6 | Wed Jan 01 23:00:00 2020 | 608.00 + 6 | Thu Jan 02 00:00:00 2020 | 608.00 + 6 | Thu Jan 02 01:00:00 2020 | 608.00 + 6 | Thu Jan 02 02:00:00 2020 | 122.00 + 7 | Wed Jan 01 20:00:00 2020 | 616.00 + 7 | Wed Jan 01 21:00:00 2020 | 616.00 + 7 | Wed Jan 01 22:00:00 2020 | 616.00 + 7 | Wed Jan 01 23:00:00 2020 | 616.00 + 7 | Thu Jan 02 00:00:00 2020 | 616.00 + 7 | Thu Jan 02 01:00:00 2020 | 616.00 + 7 | Thu Jan 02 02:00:00 2020 | 124.00 + 8 | Wed Jan 01 20:00:00 2020 | 624.00 + 8 | Wed Jan 01 21:00:00 2020 | 624.00 + 8 | Wed Jan 01 22:00:00 2020 | 624.00 + 8 | Wed Jan 01 23:00:00 2020 | 624.00 + 8 | Thu Jan 02 00:00:00 2020 | 624.00 + 8 | Thu Jan 02 01:00:00 2020 | 624.00 + 8 | Thu Jan 02 02:00:00 2020 | 126.00 + 9 | Wed Jan 01 20:00:00 2020 | 632.00 + 9 | Wed Jan 01 21:00:00 2020 | 632.00 + 9 | Wed Jan 01 22:00:00 2020 | 632.00 + 9 | Wed Jan 01 23:00:00 2020 | 632.00 + 9 | Thu Jan 02 00:00:00 2020 | 632.00 + 9 | Thu Jan 02 01:00:00 2020 | 632.00 + 9 | Thu Jan 02 02:00:00 2020 | 128.00 +(70 rows) + +DROP MATERIALIZED VIEW mv; +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 560.00 + 0 | Wed Jan 01 21:00:00 2020 | 560.00 + 0 | Wed Jan 01 22:00:00 2020 | 560.00 + 0 | Wed Jan 01 23:00:00 2020 | 560.00 + 0 | Thu Jan 02 00:00:00 2020 | 560.00 + 0 | Thu Jan 02 01:00:00 2020 | 560.00 + 0 | Thu Jan 02 02:00:00 2020 | 110.00 + 1 | Wed Jan 01 20:00:00 2020 | 568.00 + 1 | Wed Jan 01 21:00:00 2020 | 568.00 + 1 | Wed Jan 01 22:00:00 2020 | 568.00 + 1 | Wed Jan 01 23:00:00 2020 | 568.00 + 1 | Thu Jan 02 00:00:00 2020 | 568.00 + 1 | Thu Jan 02 01:00:00 2020 | 568.00 + 1 | Thu Jan 02 02:00:00 2020 | 112.00 + 2 | Wed Jan 01 20:00:00 2020 | 576.00 + 2 | Wed Jan 01 21:00:00 2020 | 576.00 + 2 | Wed Jan 01 22:00:00 2020 | 576.00 + 2 | Wed Jan 01 23:00:00 2020 | 576.00 + 2 | Thu Jan 02 00:00:00 2020 | 576.00 + 2 | Thu Jan 02 01:00:00 2020 | 576.00 + 2 | Thu Jan 02 02:00:00 2020 | 114.00 + 3 | Wed Jan 01 20:00:00 2020 | 584.00 + 3 | Wed Jan 01 21:00:00 2020 | 584.00 + 3 | Wed Jan 01 22:00:00 2020 | 584.00 + 3 | Wed Jan 01 23:00:00 2020 | 584.00 + 3 | Thu Jan 02 00:00:00 2020 | 584.00 + 3 | Thu Jan 02 01:00:00 2020 | 584.00 + 3 | Thu Jan 02 02:00:00 2020 | 116.00 + 4 | Wed Jan 01 20:00:00 2020 | 592.00 + 4 | Wed Jan 01 21:00:00 2020 | 592.00 + 4 | Wed Jan 01 22:00:00 2020 | 592.00 + 4 | Wed Jan 01 23:00:00 2020 | 592.00 + 4 | Thu Jan 02 00:00:00 2020 | 592.00 + 4 | Thu Jan 02 01:00:00 2020 | 592.00 + 4 | Thu Jan 02 02:00:00 2020 | 118.00 + 5 | Wed Jan 01 20:00:00 2020 | 600.00 + 5 | Wed Jan 01 21:00:00 2020 | 600.00 + 5 | Wed Jan 01 22:00:00 2020 | 600.00 + 5 | Wed Jan 01 23:00:00 2020 | 600.00 + 5 | Thu Jan 02 00:00:00 2020 | 600.00 + 5 | Thu Jan 02 01:00:00 2020 | 600.00 + 5 | Thu Jan 02 02:00:00 2020 | 120.00 + 6 | Wed Jan 01 20:00:00 2020 | 608.00 + 6 | Wed Jan 01 21:00:00 2020 | 608.00 + 6 | Wed Jan 01 22:00:00 2020 | 608.00 + 6 | Wed Jan 01 23:00:00 2020 | 608.00 + 6 | Thu Jan 02 00:00:00 2020 | 608.00 + 6 | Thu Jan 02 01:00:00 2020 | 608.00 + 6 | Thu Jan 02 02:00:00 2020 | 122.00 + 7 | Wed Jan 01 20:00:00 2020 | 616.00 + 7 | Wed Jan 01 21:00:00 2020 | 616.00 + 7 | Wed Jan 01 22:00:00 2020 | 616.00 + 7 | Wed Jan 01 23:00:00 2020 | 616.00 + 7 | Thu Jan 02 00:00:00 2020 | 616.00 + 7 | Thu Jan 02 01:00:00 2020 | 616.00 + 7 | Thu Jan 02 02:00:00 2020 | 124.00 + 8 | Wed Jan 01 20:00:00 2020 | 624.00 + 8 | Wed Jan 01 21:00:00 2020 | 624.00 + 8 | Wed Jan 01 22:00:00 2020 | 624.00 + 8 | Wed Jan 01 23:00:00 2020 | 624.00 + 8 | Thu Jan 02 00:00:00 2020 | 624.00 + 8 | Thu Jan 02 01:00:00 2020 | 624.00 + 8 | Thu Jan 02 02:00:00 2020 | 126.00 + 9 | Wed Jan 01 20:00:00 2020 | 632.00 + 9 | Wed Jan 01 21:00:00 2020 | 632.00 + 9 | Wed Jan 01 22:00:00 2020 | 632.00 + 9 | Wed Jan 01 23:00:00 2020 | 632.00 + 9 | Thu Jan 02 00:00:00 2020 | 632.00 + 9 | Thu Jan 02 01:00:00 2020 | 632.00 + 9 | Thu Jan 02 02:00:00 2020 | 128.00 +(70 rows) + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(13000, 14000) v; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 560.00 + 0 | Wed Jan 01 21:00:00 2020 | 560.00 + 0 | Wed Jan 01 22:00:00 2020 | 560.00 + 0 | Wed Jan 01 23:00:00 2020 | 560.00 + 0 | Thu Jan 02 00:00:00 2020 | 560.00 + 0 | Thu Jan 02 01:00:00 2020 | 560.00 + 0 | Thu Jan 02 02:00:00 2020 | 82.50 + 1 | Wed Jan 01 20:00:00 2020 | 568.00 + 1 | Wed Jan 01 21:00:00 2020 | 568.00 + 1 | Wed Jan 01 22:00:00 2020 | 568.00 + 1 | Wed Jan 01 23:00:00 2020 | 568.00 + 1 | Thu Jan 02 00:00:00 2020 | 568.00 + 1 | Thu Jan 02 01:00:00 2020 | 568.00 + 1 | Thu Jan 02 02:00:00 2020 | 79.71 + 2 | Wed Jan 01 20:00:00 2020 | 576.00 + 2 | Wed Jan 01 21:00:00 2020 | 576.00 + 2 | Wed Jan 01 22:00:00 2020 | 576.00 + 2 | Wed Jan 01 23:00:00 2020 | 576.00 + 2 | Thu Jan 02 00:00:00 2020 | 576.00 + 2 | Thu Jan 02 01:00:00 2020 | 576.00 + 2 | Thu Jan 02 02:00:00 2020 | 81.18 + 3 | Wed Jan 01 20:00:00 2020 | 584.00 + 3 | Wed Jan 01 21:00:00 2020 | 584.00 + 3 | Wed Jan 01 22:00:00 2020 | 584.00 + 3 | Wed Jan 01 23:00:00 2020 | 584.00 + 3 | Thu Jan 02 00:00:00 2020 | 584.00 + 3 | Thu Jan 02 01:00:00 2020 | 584.00 + 3 | Thu Jan 02 02:00:00 2020 | 82.65 + 4 | Wed Jan 01 20:00:00 2020 | 592.00 + 4 | Wed Jan 01 21:00:00 2020 | 592.00 + 4 | Wed Jan 01 22:00:00 2020 | 592.00 + 4 | Wed Jan 01 23:00:00 2020 | 592.00 + 4 | Thu Jan 02 00:00:00 2020 | 592.00 + 4 | Thu Jan 02 01:00:00 2020 | 592.00 + 4 | Thu Jan 02 02:00:00 2020 | 84.12 + 5 | Wed Jan 01 20:00:00 2020 | 600.00 + 5 | Wed Jan 01 21:00:00 2020 | 600.00 + 5 | Wed Jan 01 22:00:00 2020 | 600.00 + 5 | Wed Jan 01 23:00:00 2020 | 600.00 + 5 | Thu Jan 02 00:00:00 2020 | 600.00 + 5 | Thu Jan 02 01:00:00 2020 | 600.00 + 5 | Thu Jan 02 02:00:00 2020 | 85.59 + 6 | Wed Jan 01 20:00:00 2020 | 608.00 + 6 | Wed Jan 01 21:00:00 2020 | 608.00 + 6 | Wed Jan 01 22:00:00 2020 | 608.00 + 6 | Wed Jan 01 23:00:00 2020 | 608.00 + 6 | Thu Jan 02 00:00:00 2020 | 608.00 + 6 | Thu Jan 02 01:00:00 2020 | 608.00 + 6 | Thu Jan 02 02:00:00 2020 | 87.06 + 7 | Wed Jan 01 20:00:00 2020 | 616.00 + 7 | Wed Jan 01 21:00:00 2020 | 616.00 + 7 | Wed Jan 01 22:00:00 2020 | 616.00 + 7 | Wed Jan 01 23:00:00 2020 | 616.00 + 7 | Thu Jan 02 00:00:00 2020 | 616.00 + 7 | Thu Jan 02 01:00:00 2020 | 616.00 + 7 | Thu Jan 02 02:00:00 2020 | 88.53 + 8 | Wed Jan 01 20:00:00 2020 | 624.00 + 8 | Wed Jan 01 21:00:00 2020 | 624.00 + 8 | Wed Jan 01 22:00:00 2020 | 624.00 + 8 | Wed Jan 01 23:00:00 2020 | 624.00 + 8 | Thu Jan 02 00:00:00 2020 | 624.00 + 8 | Thu Jan 02 01:00:00 2020 | 624.00 + 8 | Thu Jan 02 02:00:00 2020 | 90.00 + 9 | Wed Jan 01 20:00:00 2020 | 632.00 + 9 | Wed Jan 01 21:00:00 2020 | 632.00 + 9 | Wed Jan 01 22:00:00 2020 | 632.00 + 9 | Wed Jan 01 23:00:00 2020 | 632.00 + 9 | Thu Jan 02 00:00:00 2020 | 632.00 + 9 | Thu Jan 02 01:00:00 2020 | 632.00 + 9 | Thu Jan 02 02:00:00 2020 | 91.47 +(70 rows) + +DELETE FROM events WHERE b < 100; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 560.00 + 0 | Wed Jan 01 21:00:00 2020 | 560.00 + 0 | Wed Jan 01 22:00:00 2020 | 560.00 + 0 | Wed Jan 01 23:00:00 2020 | 560.00 + 0 | Thu Jan 02 00:00:00 2020 | 560.00 + 0 | Thu Jan 02 01:00:00 2020 | 560.00 + 0 | Thu Jan 02 02:00:00 2020 | 82.50 + 1 | Wed Jan 01 20:00:00 2020 | 568.00 + 1 | Wed Jan 01 21:00:00 2020 | 568.00 + 1 | Wed Jan 01 22:00:00 2020 | 568.00 + 1 | Wed Jan 01 23:00:00 2020 | 568.00 + 1 | Thu Jan 02 00:00:00 2020 | 568.00 + 1 | Thu Jan 02 01:00:00 2020 | 568.00 + 1 | Thu Jan 02 02:00:00 2020 | 79.71 + 2 | Wed Jan 01 20:00:00 2020 | 576.00 + 2 | Wed Jan 01 21:00:00 2020 | 576.00 + 2 | Wed Jan 01 22:00:00 2020 | 576.00 + 2 | Wed Jan 01 23:00:00 2020 | 576.00 + 2 | Thu Jan 02 00:00:00 2020 | 576.00 + 2 | Thu Jan 02 01:00:00 2020 | 576.00 + 2 | Thu Jan 02 02:00:00 2020 | 81.18 + 3 | Wed Jan 01 20:00:00 2020 | 584.00 + 3 | Wed Jan 01 21:00:00 2020 | 584.00 + 3 | Wed Jan 01 22:00:00 2020 | 584.00 + 3 | Wed Jan 01 23:00:00 2020 | 584.00 + 3 | Thu Jan 02 00:00:00 2020 | 584.00 + 3 | Thu Jan 02 01:00:00 2020 | 584.00 + 3 | Thu Jan 02 02:00:00 2020 | 82.65 + 4 | Wed Jan 01 20:00:00 2020 | 592.00 + 4 | Wed Jan 01 21:00:00 2020 | 592.00 + 4 | Wed Jan 01 22:00:00 2020 | 592.00 + 4 | Wed Jan 01 23:00:00 2020 | 592.00 + 4 | Thu Jan 02 00:00:00 2020 | 592.00 + 4 | Thu Jan 02 01:00:00 2020 | 592.00 + 4 | Thu Jan 02 02:00:00 2020 | 84.12 + 5 | Wed Jan 01 20:00:00 2020 | 600.00 + 5 | Wed Jan 01 21:00:00 2020 | 600.00 + 5 | Wed Jan 01 22:00:00 2020 | 600.00 + 5 | Wed Jan 01 23:00:00 2020 | 600.00 + 5 | Thu Jan 02 00:00:00 2020 | 600.00 + 5 | Thu Jan 02 01:00:00 2020 | 600.00 + 5 | Thu Jan 02 02:00:00 2020 | 85.59 + 6 | Wed Jan 01 20:00:00 2020 | 608.00 + 6 | Wed Jan 01 21:00:00 2020 | 608.00 + 6 | Wed Jan 01 22:00:00 2020 | 608.00 + 6 | Wed Jan 01 23:00:00 2020 | 608.00 + 6 | Thu Jan 02 00:00:00 2020 | 608.00 + 6 | Thu Jan 02 01:00:00 2020 | 608.00 + 6 | Thu Jan 02 02:00:00 2020 | 87.06 + 7 | Wed Jan 01 20:00:00 2020 | 616.00 + 7 | Wed Jan 01 21:00:00 2020 | 616.00 + 7 | Wed Jan 01 22:00:00 2020 | 616.00 + 7 | Wed Jan 01 23:00:00 2020 | 616.00 + 7 | Thu Jan 02 00:00:00 2020 | 616.00 + 7 | Thu Jan 02 01:00:00 2020 | 616.00 + 7 | Thu Jan 02 02:00:00 2020 | 88.53 + 8 | Wed Jan 01 20:00:00 2020 | 624.00 + 8 | Wed Jan 01 21:00:00 2020 | 624.00 + 8 | Wed Jan 01 22:00:00 2020 | 624.00 + 8 | Wed Jan 01 23:00:00 2020 | 624.00 + 8 | Thu Jan 02 00:00:00 2020 | 624.00 + 8 | Thu Jan 02 01:00:00 2020 | 624.00 + 8 | Thu Jan 02 02:00:00 2020 | 90.00 + 9 | Wed Jan 01 20:00:00 2020 | 632.00 + 9 | Wed Jan 01 21:00:00 2020 | 632.00 + 9 | Wed Jan 01 22:00:00 2020 | 632.00 + 9 | Wed Jan 01 23:00:00 2020 | 632.00 + 9 | Thu Jan 02 00:00:00 2020 | 632.00 + 9 | Thu Jan 02 01:00:00 2020 | 632.00 + 9 | Thu Jan 02 02:00:00 2020 | 91.47 +(70 rows) + +UPDATE events SET b = b + b; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 560.00 + 0 | Wed Jan 01 21:00:00 2020 | 560.00 + 0 | Wed Jan 01 22:00:00 2020 | 560.00 + 0 | Wed Jan 01 23:00:00 2020 | 560.00 + 0 | Thu Jan 02 00:00:00 2020 | 560.00 + 0 | Thu Jan 02 01:00:00 2020 | 560.00 + 0 | Thu Jan 02 02:00:00 2020 | 82.50 + 1 | Wed Jan 01 20:00:00 2020 | 568.00 + 1 | Wed Jan 01 21:00:00 2020 | 568.00 + 1 | Wed Jan 01 22:00:00 2020 | 568.00 + 1 | Wed Jan 01 23:00:00 2020 | 568.00 + 1 | Thu Jan 02 00:00:00 2020 | 568.00 + 1 | Thu Jan 02 01:00:00 2020 | 568.00 + 1 | Thu Jan 02 02:00:00 2020 | 79.71 + 2 | Wed Jan 01 20:00:00 2020 | 576.00 + 2 | Wed Jan 01 21:00:00 2020 | 576.00 + 2 | Wed Jan 01 22:00:00 2020 | 576.00 + 2 | Wed Jan 01 23:00:00 2020 | 576.00 + 2 | Thu Jan 02 00:00:00 2020 | 576.00 + 2 | Thu Jan 02 01:00:00 2020 | 576.00 + 2 | Thu Jan 02 02:00:00 2020 | 81.18 + 3 | Wed Jan 01 20:00:00 2020 | 584.00 + 3 | Wed Jan 01 21:00:00 2020 | 584.00 + 3 | Wed Jan 01 22:00:00 2020 | 584.00 + 3 | Wed Jan 01 23:00:00 2020 | 584.00 + 3 | Thu Jan 02 00:00:00 2020 | 584.00 + 3 | Thu Jan 02 01:00:00 2020 | 584.00 + 3 | Thu Jan 02 02:00:00 2020 | 82.65 + 4 | Wed Jan 01 20:00:00 2020 | 592.00 + 4 | Wed Jan 01 21:00:00 2020 | 592.00 + 4 | Wed Jan 01 22:00:00 2020 | 592.00 + 4 | Wed Jan 01 23:00:00 2020 | 592.00 + 4 | Thu Jan 02 00:00:00 2020 | 592.00 + 4 | Thu Jan 02 01:00:00 2020 | 592.00 + 4 | Thu Jan 02 02:00:00 2020 | 84.12 + 5 | Wed Jan 01 20:00:00 2020 | 600.00 + 5 | Wed Jan 01 21:00:00 2020 | 600.00 + 5 | Wed Jan 01 22:00:00 2020 | 600.00 + 5 | Wed Jan 01 23:00:00 2020 | 600.00 + 5 | Thu Jan 02 00:00:00 2020 | 600.00 + 5 | Thu Jan 02 01:00:00 2020 | 600.00 + 5 | Thu Jan 02 02:00:00 2020 | 85.59 + 6 | Wed Jan 01 20:00:00 2020 | 608.00 + 6 | Wed Jan 01 21:00:00 2020 | 608.00 + 6 | Wed Jan 01 22:00:00 2020 | 608.00 + 6 | Wed Jan 01 23:00:00 2020 | 608.00 + 6 | Thu Jan 02 00:00:00 2020 | 608.00 + 6 | Thu Jan 02 01:00:00 2020 | 608.00 + 6 | Thu Jan 02 02:00:00 2020 | 87.06 + 7 | Wed Jan 01 20:00:00 2020 | 616.00 + 7 | Wed Jan 01 21:00:00 2020 | 616.00 + 7 | Wed Jan 01 22:00:00 2020 | 616.00 + 7 | Wed Jan 01 23:00:00 2020 | 616.00 + 7 | Thu Jan 02 00:00:00 2020 | 616.00 + 7 | Thu Jan 02 01:00:00 2020 | 616.00 + 7 | Thu Jan 02 02:00:00 2020 | 88.53 + 8 | Wed Jan 01 20:00:00 2020 | 624.00 + 8 | Wed Jan 01 21:00:00 2020 | 624.00 + 8 | Wed Jan 01 22:00:00 2020 | 624.00 + 8 | Wed Jan 01 23:00:00 2020 | 624.00 + 8 | Thu Jan 02 00:00:00 2020 | 624.00 + 8 | Thu Jan 02 01:00:00 2020 | 624.00 + 8 | Thu Jan 02 02:00:00 2020 | 90.00 + 9 | Wed Jan 01 20:00:00 2020 | 632.00 + 9 | Wed Jan 01 21:00:00 2020 | 632.00 + 9 | Wed Jan 01 22:00:00 2020 | 632.00 + 9 | Wed Jan 01 23:00:00 2020 | 632.00 + 9 | Thu Jan 02 00:00:00 2020 | 632.00 + 9 | Thu Jan 02 01:00:00 2020 | 632.00 + 9 | Thu Jan 02 02:00:00 2020 | 91.47 +(70 rows) + +REFRESH MATERIALIZED VIEW mv WITH NO DATA; +SELECT * FROM mv; + a | d_hour | avg_b +--------------------------------------------------------------------- +(0 rows) + +REFRESH MATERIALIZED VIEW mv; +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + a | d_hour | avg_b +--------------------------------------------------------------------- + 0 | Wed Jan 01 20:00:00 2020 | 1120.00 + 0 | Wed Jan 01 21:00:00 2020 | 1120.00 + 0 | Wed Jan 01 22:00:00 2020 | 1120.00 + 0 | Wed Jan 01 23:00:00 2020 | 1120.00 + 0 | Thu Jan 02 00:00:00 2020 | 1120.00 + 0 | Thu Jan 02 01:00:00 2020 | 1120.00 + 0 | Thu Jan 02 02:00:00 2020 | 280.00 + 1 | Wed Jan 01 20:00:00 2020 | 1136.00 + 1 | Wed Jan 01 21:00:00 2020 | 1136.00 + 1 | Wed Jan 01 22:00:00 2020 | 1136.00 + 1 | Wed Jan 01 23:00:00 2020 | 1136.00 + 1 | Thu Jan 02 00:00:00 2020 | 1136.00 + 1 | Thu Jan 02 01:00:00 2020 | 1136.00 + 1 | Thu Jan 02 02:00:00 2020 | 284.00 + 2 | Wed Jan 01 20:00:00 2020 | 1152.00 + 2 | Wed Jan 01 21:00:00 2020 | 1152.00 + 2 | Wed Jan 01 22:00:00 2020 | 1152.00 + 2 | Wed Jan 01 23:00:00 2020 | 1152.00 + 2 | Thu Jan 02 00:00:00 2020 | 1152.00 + 2 | Thu Jan 02 01:00:00 2020 | 1152.00 + 2 | Thu Jan 02 02:00:00 2020 | 288.00 + 3 | Wed Jan 01 20:00:00 2020 | 1168.00 + 3 | Wed Jan 01 21:00:00 2020 | 1168.00 + 3 | Wed Jan 01 22:00:00 2020 | 1168.00 + 3 | Wed Jan 01 23:00:00 2020 | 1168.00 + 3 | Thu Jan 02 00:00:00 2020 | 1168.00 + 3 | Thu Jan 02 01:00:00 2020 | 1168.00 + 3 | Thu Jan 02 02:00:00 2020 | 292.00 + 4 | Wed Jan 01 20:00:00 2020 | 1184.00 + 4 | Wed Jan 01 21:00:00 2020 | 1184.00 + 4 | Wed Jan 01 22:00:00 2020 | 1184.00 + 4 | Wed Jan 01 23:00:00 2020 | 1184.00 + 4 | Thu Jan 02 00:00:00 2020 | 1184.00 + 4 | Thu Jan 02 01:00:00 2020 | 1184.00 + 4 | Thu Jan 02 02:00:00 2020 | 296.00 + 5 | Wed Jan 01 20:00:00 2020 | 1200.00 + 5 | Wed Jan 01 21:00:00 2020 | 1200.00 + 5 | Wed Jan 01 22:00:00 2020 | 1200.00 + 5 | Wed Jan 01 23:00:00 2020 | 1200.00 + 5 | Thu Jan 02 00:00:00 2020 | 1200.00 + 5 | Thu Jan 02 01:00:00 2020 | 1200.00 + 5 | Thu Jan 02 02:00:00 2020 | 300.00 + 6 | Wed Jan 01 20:00:00 2020 | 1216.00 + 6 | Wed Jan 01 21:00:00 2020 | 1216.00 + 6 | Wed Jan 01 22:00:00 2020 | 1216.00 + 6 | Wed Jan 01 23:00:00 2020 | 1216.00 + 6 | Thu Jan 02 00:00:00 2020 | 1216.00 + 6 | Thu Jan 02 01:00:00 2020 | 1216.00 + 6 | Thu Jan 02 02:00:00 2020 | 304.00 + 7 | Wed Jan 01 20:00:00 2020 | 1232.00 + 7 | Wed Jan 01 21:00:00 2020 | 1232.00 + 7 | Wed Jan 01 22:00:00 2020 | 1232.00 + 7 | Wed Jan 01 23:00:00 2020 | 1232.00 + 7 | Thu Jan 02 00:00:00 2020 | 1232.00 + 7 | Thu Jan 02 01:00:00 2020 | 1232.00 + 7 | Thu Jan 02 02:00:00 2020 | 308.00 + 8 | Wed Jan 01 20:00:00 2020 | 1248.00 + 8 | Wed Jan 01 21:00:00 2020 | 1248.00 + 8 | Wed Jan 01 22:00:00 2020 | 1248.00 + 8 | Wed Jan 01 23:00:00 2020 | 1248.00 + 8 | Thu Jan 02 00:00:00 2020 | 1248.00 + 8 | Thu Jan 02 01:00:00 2020 | 1248.00 + 8 | Thu Jan 02 02:00:00 2020 | 312.00 + 9 | Wed Jan 01 20:00:00 2020 | 1264.00 + 9 | Wed Jan 01 21:00:00 2020 | 1264.00 + 9 | Wed Jan 01 22:00:00 2020 | 1264.00 + 9 | Wed Jan 01 23:00:00 2020 | 1264.00 + 9 | Thu Jan 02 00:00:00 2020 | 1264.00 + 9 | Thu Jan 02 01:00:00 2020 | 1264.00 + 9 | Thu Jan 02 02:00:00 2020 | 316.00 +(70 rows) + +DROP MATERIALIZED VIEW mv; +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour WITH NO DATA; +SELECT * FROM mv; + a | d_hour | avg_b +--------------------------------------------------------------------- +(0 rows) + +DROP MATERIALIZED VIEW mv; +SET client_min_messages TO WARNING; -- suppress cascade messages +DROP SCHEMA cimv CASCADE; diff --git a/src/test/regress/expected/multi_extension.out b/src/test/regress/expected/multi_extension.out index b640a8aa3..8efc65d1c 100644 --- a/src/test/regress/expected/multi_extension.out +++ b/src/test/regress/expected/multi_extension.out @@ -448,10 +448,19 @@ SELECT * FROM print_extension_changes(); function task_tracker_task_status(bigint,integer) | function worker_execute_sql_task(bigint,integer,text,boolean) | function worker_merge_files_and_run_query(bigint,integer,text,text) | + | function combine_agg(oid,bytea) + | function combine_agg_ffunc(internal,oid,bytea,anyelement) + | function combine_agg_sfunc(internal,oid,bytea) + | function combine_agg_sfunc(internal,oid,bytea,anyelement) | function create_citus_local_table(regclass) + | function finalize_agg(oid,bytea,anyelement) + | function partial_agg(oid,anyelement,integer) + | function partial_agg_ffunc(internal) + | function partial_agg_sfunc(internal,oid,anyelement,integer) | function undistribute_table(regclass) | function worker_record_sequence_dependency(regclass,regclass,name) -(10 rows) + | table pg_cimv +(19 rows) DROP TABLE prev_objects, extension_diff; -- show running version diff --git a/src/test/regress/multi_schedule b/src/test/regress/multi_schedule index 57ac0f47a..596ab1abd 100644 --- a/src/test/regress/multi_schedule +++ b/src/test/regress/multi_schedule @@ -96,7 +96,7 @@ test: multi_subquery_in_where_reference_clause join geqo adaptive_executor propa test: multi_subquery_union multi_subquery_in_where_clause multi_subquery_misc statement_cancel_error_message test: multi_agg_distinct multi_agg_approximate_distinct multi_limit_clause_approximate multi_outer_join_reference multi_single_relation_subquery multi_prepare_plsql set_role_in_transaction test: multi_reference_table multi_select_for_update relation_access_tracking pg13_with_ties -test: custom_aggregate_support aggregate_support tdigest_aggregate_support +test: custom_aggregate_support aggregate_support tdigest_aggregate_support cimv test: multi_average_expression multi_working_columns multi_having_pushdown having_subquery test: multi_array_agg multi_limit_clause multi_orderby_limit_pushdown test: multi_jsonb_agg multi_jsonb_object_agg multi_json_agg multi_json_object_agg bool_agg ch_bench_having chbenchmark_all_queries expression_reference_join anonymous_columns diff --git a/src/test/regress/sql/cimv.sql b/src/test/regress/sql/cimv.sql new file mode 100644 index 000000000..dc4d65496 --- /dev/null +++ b/src/test/regress/sql/cimv.sql @@ -0,0 +1,360 @@ +-- +-- CIMV +-- Tests for Citus Incremental Materialized Views +-- + +\set VERBOSITY terse +SET citus.next_shard_id TO 400000; +CREATE SCHEMA cimv; +SET search_path TO cimv, public; + +SET citus.shard_count TO 4; + +CREATE TABLE events (a int, b int, c double precision, d timestamp, e bigint); + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(1, 10000) v; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + min(b) AS min_b, + max(b) AS max_b, + avg(b) AS avg_b, + min(c) AS min_c, + max(c) AS max_c, + avg(c) AS avg_c, + min(e) AS min_e, + max(e) AS max_e, + avg(e) AS avg_e +FROM events +WHERE b > 10 +GROUP BY a, d_hour; + +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(10000, 11000) v; + +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DELETE FROM events WHERE b < 100; + +DROP VIEW mv; + +DROP MATERIALIZED VIEW mv; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DELETE FROM events WHERE b < 20; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +UPDATE events SET b = b + b; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DROP MATERIALIZED VIEW mv; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(11000, 12000) v; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DELETE FROM events WHERE b < 100; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +UPDATE events SET b = b + b; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +REFRESH MATERIALIZED VIEW mv WITH NO DATA; + +SELECT * FROM mv; + +REFRESH MATERIALIZED VIEW mv; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + + +DROP MATERIALIZED VIEW mv; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour WITH NO DATA; + +SELECT * FROM mv; + +DROP MATERIALIZED VIEW mv; + +SELECT create_distributed_table('events', 'a'); + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + min(b) AS min_b, + max(b) AS max_b, + avg(b) AS avg_b, + min(c) AS min_c, + max(c) AS max_c, + avg(c) AS avg_c, + min(e) AS min_e, + max(e) AS max_e, + avg(e) AS avg_e +FROM events +WHERE b > 10 +GROUP BY a, d_hour; + +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(12000, 13000) v; + +SELECT a, + d_hour, + min_b::numeric(12,2), + max_b::numeric(12,2), + avg_b::numeric(12,2), + min_c::numeric(12,2), + max_c::numeric(12,2), + avg_c::numeric(12,2), + min_e::numeric(12,2), + max_e::numeric(12,2), + avg_e::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DELETE FROM events WHERE b < 100; + +DROP VIEW mv; + +DROP MATERIALIZED VIEW mv; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DELETE FROM events WHERE b < 20; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + + +UPDATE events SET b = b + b; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + + +DROP MATERIALIZED VIEW mv; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +INSERT INTO events +SELECT v % 10 AS a, + v % 100 AS b, + v / 3.0 AS c, + timestamp '2020-01-01 20:00:00' + + ((v / 10000.0) * (timestamp '2020-01-01 15:00:00' - + timestamp '2020-01-01 10:00:00')) AS d, + v AS e +FROM generate_series(13000, 14000) v; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DELETE FROM events WHERE b < 100; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +UPDATE events SET b = b + b; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +REFRESH MATERIALIZED VIEW mv WITH NO DATA; + +SELECT * FROM mv; + +REFRESH MATERIALIZED VIEW mv; + +SELECT a, + d_hour, + avg_b::numeric(12,2) +FROM mv +ORDER BY a, d_hour; + +DROP MATERIALIZED VIEW mv; + +CREATE MATERIALIZED VIEW mv WITH (citus.cimv, citus.insertonlycapture) AS +SELECT a, + date_trunc('hour', d) AS d_hour, + avg(b) AS avg_b +FROM events +WHERE b > 10 +GROUP BY a, d_hour WITH NO DATA; + +SELECT * FROM mv; + +DROP MATERIALIZED VIEW mv; + +SET client_min_messages TO WARNING; -- suppress cascade messages +DROP SCHEMA cimv CASCADE;