>>>inferred_schema.graphqls
"An RFC-3339 compliant Full Date Scalar"
scalar Date

"A DateTime scalar that handles both full RFC3339 and shorter timestamp formats"
scalar DateTime

type EventLikeCount {
  eventid: Long!
  num: Long!
  test: Long!
}

type Events {
  url: String!
  date: String!
  time: String!
  title: String!
  abstract: String!
  location: String!
  speakers: [Events_speakersOutput]!
  last_updated: DateTime!
  id: Long!
  full_text: String!
  startTime: String
  startTimestamp: DateTime
  likeCount(limit: Int = 10, offset: Int = 0): [EventLikeCount!]
}

type Events_speakersOutput {
  name: String
  title: String
  company: String
}

"A JSON scalar"
scalar JSON

"24-hour clock time value string in the format `hh:mm:ss` or `hh:mm:ss.sss`."
scalar LocalTime

"A 64-bit signed integer"
scalar Long

type PersonalizedEventSearch {
  url: String!
  date: String!
  time: String!
  title: String!
  abstract: String!
  location: String!
  speakers: [PersonalizedEventSearch_speakersOutput]!
  last_updated: DateTime!
  id: Long!
  full_text: String!
  startTime: String
  startTimestamp: DateTime
  score: Float!
}

type PersonalizedEventSearch_speakersOutput {
  name: String
  title: String
  company: String
}

type Query {
  Events(id: Long!, limit: Int = 10, offset: Int = 0): [Events!]
  EventsLiked(userid: String!, limit: Int = 10, offset: Int = 0): [Events!]
  PersonalizedEventSearch(query: String!, userid: String!, limit: Int = 10, offset: Int = 0): [PersonalizedEventSearch!]
  RecommendedEvents(userid: String!, limit: Int = 10, offset: Int = 0): [RecommendedEvents!]
}

type RecommendedEvents {
  url: String!
  date: String!
  time: String!
  title: String!
  abstract: String!
  location: String!
  speakers: [RecommendedEvents_speakersOutput]!
  last_updated: DateTime!
  id: Long!
  full_text: String!
  startTime: String
  startTimestamp: DateTime
  score: Float
}

type RecommendedEvents_speakersOutput {
  name: String
  title: String
  company: String
}

enum _McpMethodType {
  NONE
  TOOL
  RESOURCE
}

enum _RestMethodType {
  NONE
  GET
  POST
}

directive @api(mcp: _McpMethodType, rest: _RestMethodType, uri: String) on QUERY | MUTATION | FIELD_DEFINITION

>>>pipeline_explain.txt
=== EventLikeCount
ID:          default_catalog.default_database.EventLikeCount
Type:        state
Stage:       flink
Primary key: eventid
Timestamp:   -
Row count:   ~3e6
---
Schema:
 - eventid: BIGINT NOT NULL
 - num: BIGINT NOT NULL
 - test: BIGINT NOT NULL
Inputs:
 - default_catalog.default_database._UserLikes

=== Events
ID:          default_catalog.default_database.Events
Type:        state
Stage:       flink
Primary key: id
Timestamp:   last_updated
Row count:   ~2e7
---
Schema:
 - url: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - date: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - time: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - title: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - abstract: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - location: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - speakers: RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" title, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" company) NOT NULL ARRAY NOT NULL
 - last_updated: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
 - id: BIGINT NOT NULL
 - full_text: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - startTime: VARCHAR(2147483647) CHARACTER SET "UTF-16LE"
 - embedding: RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD')
 - startTimestamp: TIMESTAMP(3)
Inputs:
 - default_catalog.default_database._ProcessedEventStream
Annotations:
 - mostRecentDistinct: true
 - stream-root: _EventStream

=== EventsLiked
ID:          default_catalog.default_database.EventsLiked
Type:        query
Stage:       postgres
---
Inputs:
 - default_catalog.default_database.Events
 - default_catalog.default_database._UserLikes
Annotations:
 - parameters: userid
 - base-table: Events

=== PersonalizedEventSearch
ID:          default_catalog.default_database.PersonalizedEventSearch
Type:        query
Stage:       postgres
---
Inputs:
 - default_catalog.default_database.Events
 - default_catalog.default_database._UserInterests
Annotations:
 - parameters: query, userid
 - base-table: PersonalizedEventSearch

=== RecommendedEvents
ID:          default_catalog.default_database.RecommendedEvents
Type:        query
Stage:       postgres
---
Inputs:
 - default_catalog.default_database.Events
 - default_catalog.default_database._UserInterests
Annotations:
 - parameters: userid
 - base-table: RecommendedEvents

=== _AddInterest
ID:          default_catalog.default_database._AddInterest
Type:        stream
Stage:       flink
Primary key: _uuid
Timestamp:   event_time
Row count:   ~1e8
---
Schema:
 - _uuid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - text: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - event_time: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
 - embedding: RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD')
Inputs:
 - default_catalog.default_database._AddInterestStream
Annotations:
 - stream-root: _AddInterestStream

=== _AddInterestStream
ID:          default_catalog.default_database._AddInterestStream
Type:        stream
Stage:       flink
Primary key: _uuid
Timestamp:   event_time
Row count:   ~1e8
---
Schema:
 - _uuid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - text: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - event_time: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
Inputs:
 - default_catalog.default_database._AddInterestStream__base
Annotations:
 - stream-root: _AddInterestStream

=== _EventStream
ID:          default_catalog.default_database._EventStream
Type:        stream
Stage:       flink
Primary key: url, last_updated
Timestamp:   last_updated
Row count:   ~1e8
---
Schema:
 - url: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - date: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - time: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - title: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - abstract: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - location: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - speakers: RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" title, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" company) NOT NULL ARRAY NOT NULL
 - last_updated: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
Inputs:
 - default_catalog.default_database._EventStream__base
Annotations:
 - features: DENORMALIZE (feature)
 - stream-root: _EventStream

=== _LikeVector
ID:          default_catalog.default_database._LikeVector
Type:        stream
Stage:       flink
Primary key: -
Timestamp:   event_time
Row count:   ~2e7
---
Schema:
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - embedding: RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD')
 - title: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - abstract: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - location: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - event_time: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
Inputs:
 - default_catalog.default_database.Events
 - default_catalog.default_database._Likes
Annotations:
 - stream-root: _Likes

=== _Likes
ID:          default_catalog.default_database._Likes
Type:        stream
Stage:       flink
Primary key: _uuid
Timestamp:   event_time
Row count:   ~1e8
---
Schema:
 - _uuid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - eventid: BIGINT NOT NULL
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - liked: BOOLEAN NOT NULL
 - event_time: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
Inputs:
 - default_catalog.default_database._Likes__base
Annotations:
 - stream-root: _Likes

=== _ProcessedEventStream
ID:          default_catalog.default_database._ProcessedEventStream
Type:        stream
Stage:       flink
Primary key: url, last_updated
Timestamp:   last_updated
Row count:   ~1e8
---
Schema:
 - url: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - date: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - time: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - title: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - abstract: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - location: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - speakers: RecordType:peek_no_expand(VARCHAR(2147483647) CHARACTER SET "UTF-16LE" name, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" title, VARCHAR(2147483647) CHARACTER SET "UTF-16LE" company) NOT NULL ARRAY NOT NULL
 - last_updated: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
 - id: BIGINT NOT NULL
 - full_text: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - startTime: VARCHAR(2147483647) CHARACTER SET "UTF-16LE"
 - embedding: RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD')
 - startTimestamp: TIMESTAMP(3)
Inputs:
 - default_catalog.default_database._EventStream
Annotations:
 - stream-root: _EventStream

=== _UserInterestVectors
ID:          default_catalog.default_database._UserInterestVectors
Type:        stream
Stage:       flink
Primary key: -
Timestamp:   event_time
Row count:   ~2e8
---
Schema:
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - embedding: RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD')
 - event_time: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
Inputs:
 - default_catalog.default_database._AddInterest
 - default_catalog.default_database._LikeVector

=== _UserInterests
ID:          default_catalog.default_database._UserInterests
Type:        state
Stage:       flink
Primary key: userid
Timestamp:   -
Row count:   ~1e7
---
Schema:
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - interestVector: RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD')
Inputs:
 - default_catalog.default_database._UserInterestVectors

=== _UserLikes
ID:          default_catalog.default_database._UserLikes
Type:        state
Stage:       flink
Primary key: userid, eventid
Timestamp:   event_time
Row count:   ~2e7
---
Schema:
 - _uuid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - eventid: BIGINT NOT NULL
 - userid: VARCHAR(2147483647) CHARACTER SET "UTF-16LE" NOT NULL
 - liked: BOOLEAN NOT NULL
 - event_time: TIMESTAMP_LTZ(3) *ROWTIME* NOT NULL
Inputs:
 - default_catalog.default_database._Likes
Annotations:
 - mostRecentDistinct: true
 - stream-root: _Likes

>>>flink-sql-no-functions.sql
CREATE TEMPORARY TABLE `_EventStream__schema` (
  `url` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `date` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `time` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `title` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `abstract` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `location` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `speakers` ROW(`name` VARCHAR(2147483647) CHARACTER SET `UTF-16LE`, `title` VARCHAR(2147483647) CHARACTER SET `UTF-16LE`, `company` VARCHAR(2147483647) CHARACTER SET `UTF-16LE`) NOT NULL ARRAY NOT NULL,
  `last_updated` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL
)
WITH (
  'connector' = 'datagen'
);
CREATE TABLE `_EventStream` (
  PRIMARY KEY (`url`, `last_updated`) NOT ENFORCED,
  WATERMARK FOR `last_updated` AS `last_updated` - INTERVAL '0.001' SECOND
)
WITH (
  'format' = 'json',
  'path' = '${DATA_PATH}/events.jsonl',
  'json.timestamp-format.standard' = 'ISO-8601',
  'source.monitor-interval' = '1 sec',
  'connector' = 'filesystem'
)
LIKE `_EventStream__schema`;
CREATE VIEW `_ProcessedEventStream`
AS
SELECT *, COALESCE(CAST(`REGEXP_EXTRACT`(`url`, '(\d*)$') AS BIGINT), 0) AS `id`, `concat`(`title`, '\n', `abstract`) AS `full_text`, `concat`(TRIM(BOTH ' ' FROM `REGEXP_EXTRACT`(`date`, '^[^-]*')), ' ', TRIM(BOTH ' ' FROM `REGEXP_EXTRACT`(`time`, '\d\d?:\d\d\s(AM|PM)'))) AS `startTime`
FROM `_EventStream`;
ALTER VIEW `_ProcessedEventStream`
AS
SELECT `url`, `date`, `time`, `title`, `abstract`, `location`, `speakers`, `last_updated`, `id`, `full_text`, `startTime`, `vector_embed`(`full_text`, 'text-embedding-3-small') AS `embedding`
FROM (SELECT *, COALESCE(CAST(`REGEXP_EXTRACT`(`url`, '(\d*)$') AS BIGINT), 0) AS `id`, `concat`(`title`, '\n', `abstract`) AS `full_text`, `concat`(TRIM(BOTH ' ' FROM `REGEXP_EXTRACT`(`date`, '^[^-]*')), ' ', TRIM(BOTH ' ' FROM `REGEXP_EXTRACT`(`time`, '\d\d?:\d\d\s(AM|PM)'))) AS `startTime`
  FROM `_EventStream`);
ALTER VIEW `_ProcessedEventStream`
AS
SELECT `url`, `date`, `time`, `title`, `abstract`, `location`, `speakers`, `last_updated`, `id`, `full_text`, `startTime`, `embedding`, `TO_TIMESTAMP`(`concat`(`startTime`, ' PDT'), 'MMMM d, yyyy h:mm a z') AS `startTimestamp`
FROM (SELECT `url`, `date`, `time`, `title`, `abstract`, `location`, `speakers`, `last_updated`, `id`, `full_text`, `startTime`, `vector_embed`(`full_text`, 'text-embedding-3-small') AS `embedding`
  FROM (SELECT *, COALESCE(CAST(`REGEXP_EXTRACT`(`url`, '(\d*)$') AS BIGINT), 0) AS `id`, `concat`(`title`, '\n', `abstract`) AS `full_text`, `concat`(TRIM(BOTH ' ' FROM `REGEXP_EXTRACT`(`date`, '^[^-]*')), ' ', TRIM(BOTH ' ' FROM `REGEXP_EXTRACT`(`time`, '\d\d?:\d\d\s(AM|PM)'))) AS `startTime`
    FROM `_EventStream`));
CREATE VIEW `Events`
AS
SELECT `url`, `date`, `time`, `title`, `abstract`, `location`, `speakers`, `last_updated`, `id`, `full_text`, `startTime`, `embedding`, `startTimestamp`
FROM (SELECT `url`, `date`, `time`, `title`, `abstract`, `location`, `speakers`, `last_updated`, `id`, `full_text`, `startTime`, `embedding`, `startTimestamp`, ROW_NUMBER() OVER (PARTITION BY `id` ORDER BY `last_updated` DESC NULLS LAST) AS `__sqrlinternal_rownum`
  FROM `default_catalog`.`default_database`.`_ProcessedEventStream`) AS `t`
WHERE `__sqrlinternal_rownum` = 1;
CREATE TEMPORARY TABLE `_Likes__schema` (
  `_uuid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `eventid` BIGINT NOT NULL,
  `userid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `liked` BOOLEAN NOT NULL,
  `event_time` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL
)
WITH (
  'connector' = 'datagen'
);
CREATE TABLE `_Likes` (
  PRIMARY KEY (`_uuid`) NOT ENFORCED,
  WATERMARK FOR `event_time` AS `event_time` - INTERVAL '0.001' SECOND
)
WITH (
  'format' = 'flexible-json',
  'path' = '${DATA_PATH}/likes.jsonl',
  'source.monitor-interval' = '1 sec',
  'flexible-json.timestamp-format.standard' = 'ISO-8601',
  'connector' = 'filesystem'
)
LIKE `_Likes__schema`;
CREATE TEMPORARY TABLE `_AddInterestStream__schema` (
  `_uuid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `userid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `text` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `event_time` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL
)
WITH (
  'connector' = 'datagen'
);
CREATE TABLE `_AddInterestStream` (
  PRIMARY KEY (`_uuid`) NOT ENFORCED,
  WATERMARK FOR `event_time` AS `event_time` - INTERVAL '0.001' SECOND
)
WITH (
  'format' = 'flexible-json',
  'path' = '${DATA_PATH}/add_interest.jsonl',
  'source.monitor-interval' = '1',
  'connector' = 'filesystem'
)
LIKE `_AddInterestStream__schema`;
CREATE VIEW `_AddInterest`
AS
SELECT *, `vector_embed`(`text`, 'text-embedding-3-small') AS `embedding`
FROM `_AddInterestStream`;
CREATE VIEW `_LikeVector`
AS
SELECT `l`.`userid`, `e`.`embedding`, `e`.`title`, `e`.`abstract`, `e`.`location`, `l`.`event_time`
FROM `_Likes` AS `l`
 INNER JOIN `Events` FOR SYSTEM_TIME AS OF `l`.`event_time` AS `e` ON `l`.`eventid` = `e`.`id`
WHERE `l`.`liked`;
CREATE VIEW `_UserInterestVectors`
AS
SELECT `userid`, `embedding`, `event_time`
FROM `_LikeVector`
UNION ALL
SELECT `userid`, `embedding`, `event_time`
FROM `_AddInterest`;
CREATE VIEW `_UserInterests`
AS
SELECT `userid`, `center`(`embedding`) AS `interestVector`
FROM `_UserInterestVectors`
GROUP BY `userid`;
CREATE VIEW `_UserLikes`
AS
SELECT `_uuid`, `eventid`, `userid`, `liked`, `event_time`
FROM (SELECT `_uuid`, `eventid`, `userid`, `liked`, `event_time`, ROW_NUMBER() OVER (PARTITION BY `userid`, `eventid` ORDER BY `event_time` DESC NULLS LAST) AS `__sqrlinternal_rownum`
  FROM `default_catalog`.`default_database`.`_Likes`) AS `t`
WHERE `__sqrlinternal_rownum` = 1;
CREATE VIEW `EventLikeCount`
AS
SELECT `eventid`, COUNT(*) AS `num`, AVG(`eventid`) AS `test`
FROM `_UserLikes`
WHERE `liked`
GROUP BY `eventid`;
CREATE TABLE `EventLikeCount_1` (
  `eventid` BIGINT NOT NULL,
  `num` BIGINT NOT NULL,
  `test` BIGINT NOT NULL,
  PRIMARY KEY (`eventid`) NOT ENFORCED
)
WITH (
  'connector' = 'jdbc-sqrl',
  'driver' = 'org.postgresql.Driver',
  'password' = '${POSTGRES_PASSWORD}',
  'table-name' = 'EventLikeCount',
  'url' = 'jdbc:postgresql://${POSTGRES_AUTHORITY}',
  'username' = '${POSTGRES_USERNAME}'
);
CREATE TABLE `Events_2` (
  `url` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `date` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `time` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `title` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `abstract` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `location` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `speakers` RAW('com.datasqrl.flinkrunner.stdlib.json.FlinkJsonType', 'AERjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLmpzb24uRmxpbmtKc29uVHlwZVNlcmlhbGl6ZXJTbmFwc2hvdAAAAAM='),
  `last_updated` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL,
  `id` BIGINT NOT NULL,
  `full_text` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `startTime` VARCHAR(2147483647) CHARACTER SET `UTF-16LE`,
  `embedding` RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD'),
  `startTimestamp` TIMESTAMP(3),
  PRIMARY KEY (`id`) NOT ENFORCED
)
WITH (
  'connector' = 'jdbc-sqrl',
  'driver' = 'org.postgresql.Driver',
  'password' = '${POSTGRES_PASSWORD}',
  'sink.on-conflict.action' = 'TIMESTAMP',
  'sink.on-conflict.timestamp-column' = 'last_updated',
  'table-name' = 'Events',
  'url' = 'jdbc:postgresql://${POSTGRES_AUTHORITY}',
  'username' = '${POSTGRES_USERNAME}'
);
CREATE TABLE `_UserInterests_3` (
  `userid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `interestVector` RAW('com.datasqrl.flinkrunner.stdlib.vector.FlinkVectorType', 'AEhjb20uZGF0YXNxcmwuZmxpbmtydW5uZXIuc3RkbGliLnZlY3Rvci5GbGlua1ZlY3RvclR5cGVTZXJpYWxpemVyU25hcHNob3QAAAAD'),
  PRIMARY KEY (`userid`) NOT ENFORCED
)
WITH (
  'connector' = 'jdbc-sqrl',
  'driver' = 'org.postgresql.Driver',
  'password' = '${POSTGRES_PASSWORD}',
  'table-name' = '_UserInterests',
  'url' = 'jdbc:postgresql://${POSTGRES_AUTHORITY}',
  'username' = '${POSTGRES_USERNAME}'
);
CREATE TABLE `_UserLikes_4` (
  `_uuid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `eventid` BIGINT NOT NULL,
  `userid` VARCHAR(2147483647) CHARACTER SET `UTF-16LE` NOT NULL,
  `liked` BOOLEAN NOT NULL,
  `event_time` TIMESTAMP(3) WITH LOCAL TIME ZONE NOT NULL,
  PRIMARY KEY (`userid`, `eventid`) NOT ENFORCED
)
WITH (
  'connector' = 'jdbc-sqrl',
  'driver' = 'org.postgresql.Driver',
  'password' = '${POSTGRES_PASSWORD}',
  'sink.on-conflict.action' = 'TIMESTAMP',
  'sink.on-conflict.timestamp-column' = 'event_time',
  'table-name' = '_UserLikes',
  'url' = 'jdbc:postgresql://${POSTGRES_AUTHORITY}',
  'username' = '${POSTGRES_USERNAME}'
);
EXECUTE STATEMENT SET BEGIN
INSERT INTO `default_catalog`.`default_database`.`EventLikeCount_1`
SELECT *
 FROM `default_catalog`.`default_database`.`EventLikeCount`
;
INSERT INTO `default_catalog`.`default_database`.`Events_2`
 SELECT `url`, `date`, `time`, `title`, `abstract`, `location`, `to_jsonb`(`speakers`) AS `speakers`, `last_updated`, `id`, `full_text`, `startTime`, `embedding`, `startTimestamp`
  FROM `default_catalog`.`default_database`.`_ProcessedEventStream`
 ;
 INSERT INTO `default_catalog`.`default_database`.`_UserInterests_3`
  SELECT *
   FROM `default_catalog`.`default_database`.`_UserInterests`
  ;
  INSERT INTO `default_catalog`.`default_database`.`_UserLikes_4`
   SELECT *
    FROM `default_catalog`.`default_database`.`_Likes`
   ;
   END
>>>kafka.json
{
  "topics" : [ ],
  "testRunnerTopics" : [ ]
}
>>>postgres-schema.sql
CREATE EXTENSION IF NOT EXISTS vector;

CREATE TABLE IF NOT EXISTS "EventLikeCount" ("eventid" BIGINT NOT NULL, "num" BIGINT NOT NULL, "test" BIGINT NOT NULL, PRIMARY KEY ("eventid"));
CREATE TABLE IF NOT EXISTS "Events" ("url" TEXT NOT NULL, "date" TEXT NOT NULL, "time" TEXT NOT NULL, "title" TEXT NOT NULL, "abstract" TEXT NOT NULL, "location" TEXT NOT NULL, "speakers" JSONB, "last_updated" TIMESTAMP WITH TIME ZONE NOT NULL, "id" BIGINT NOT NULL, "full_text" TEXT NOT NULL, "startTime" TEXT, "embedding" VECTOR(1536), "startTimestamp" TIMESTAMP WITHOUT TIME ZONE, PRIMARY KEY ("id"));
CREATE TABLE IF NOT EXISTS "_UserInterests" ("userid" TEXT NOT NULL, "interestVector" VECTOR(1536), PRIMARY KEY ("userid"));
CREATE TABLE IF NOT EXISTS "_UserLikes" ("_uuid" TEXT NOT NULL, "eventid" BIGINT NOT NULL, "userid" TEXT NOT NULL, "liked" BOOLEAN NOT NULL, "event_time" TIMESTAMP WITH TIME ZONE NOT NULL, PRIMARY KEY ("userid","eventid"));

CREATE INDEX IF NOT EXISTS "Events_text_c3c4" ON "Events" USING GIN (to_tsvector('english', coalesce("title", '') || ' ' || coalesce("abstract", '') ))
>>>postgres-views.sql

>>>vertx.json
{
  "models" : {
    "v1" : {
      "queries" : [
        {
          "type" : "args",
          "parentType" : "Query",
          "fieldName" : "Events",
          "exec" : {
            "arguments" : [
              {
                "type" : "variable",
                "path" : "limit"
              },
              {
                "type" : "variable",
                "path" : "id"
              },
              {
                "type" : "variable",
                "path" : "offset"
              }
            ],
            "query" : {
              "type" : "SqlQuery",
              "sql" : "SELECT *\nFROM \"Events\"\nWHERE \"id\" = $1",
              "parameters" : [
                {
                  "type" : "arg",
                  "path" : "id",
                  "sqlType" : "BIGINT"
                }
              ],
              "pagination" : "LIMIT_AND_OFFSET",
              "cacheDurationMs" : 0,
              "database" : "POSTGRES"
            }
          }
        },
        {
          "type" : "args",
          "parentType" : "Events",
          "fieldName" : "likeCount",
          "exec" : {
            "arguments" : [
              {
                "type" : "variable",
                "path" : "limit"
              },
              {
                "type" : "variable",
                "path" : "offset"
              }
            ],
            "query" : {
              "type" : "SqlQuery",
              "sql" : "SELECT *\nFROM \"EventLikeCount\"\nWHERE $1 = \"eventid\"",
              "parameters" : [
                {
                  "type" : "source",
                  "key" : "id"
                }
              ],
              "pagination" : "LIMIT_AND_OFFSET",
              "cacheDurationMs" : 0,
              "database" : "POSTGRES"
            }
          }
        },
        {
          "type" : "args",
          "parentType" : "Query",
          "fieldName" : "EventsLiked",
          "exec" : {
            "arguments" : [
              {
                "type" : "variable",
                "path" : "limit"
              },
              {
                "type" : "variable",
                "path" : "offset"
              },
              {
                "type" : "variable",
                "path" : "userid"
              }
            ],
            "query" : {
              "type" : "SqlQuery",
              "sql" : "SELECT \"Events\".\"url\", \"Events\".\"date\", \"Events\".\"time\", \"Events\".\"title\", \"Events\".\"abstract\", \"Events\".\"location\", \"Events\".\"speakers\", \"Events\".\"last_updated\", \"Events\".\"id\", \"Events\".\"full_text\", \"Events\".\"startTime\", \"Events\".\"embedding\", \"Events\".\"startTimestamp\"\nFROM \"_UserLikes\"\n INNER JOIN \"Events\" ON \"_UserLikes\".\"eventid\" = \"Events\".\"id\"\nWHERE \"_UserLikes\".\"userid\" = $1\nORDER BY \"Events\".\"startTimestamp\" NULLS FIRST",
              "parameters" : [
                {
                  "type" : "arg",
                  "path" : "userid",
                  "sqlType" : "VARCHAR"
                }
              ],
              "pagination" : "LIMIT_AND_OFFSET",
              "cacheDurationMs" : 0,
              "database" : "POSTGRES"
            }
          }
        },
        {
          "type" : "args",
          "parentType" : "Query",
          "fieldName" : "PersonalizedEventSearch",
          "exec" : {
            "arguments" : [
              {
                "type" : "variable",
                "path" : "limit"
              },
              {
                "type" : "variable",
                "path" : "offset"
              },
              {
                "type" : "variable",
                "path" : "userid"
              },
              {
                "type" : "variable",
                "path" : "query"
              }
            ],
            "query" : {
              "type" : "SqlQuery",
              "sql" : "SELECT \"Events\".\"url\", \"Events\".\"date\", \"Events\".\"time\", \"Events\".\"title\", \"Events\".\"abstract\", \"Events\".\"location\", \"Events\".\"speakers\", \"Events\".\"last_updated\", \"Events\".\"id\", \"Events\".\"full_text\", \"Events\".\"startTime\", \"Events\".\"embedding\", \"Events\".\"startTimestamp\", COALESCE(1 - (\"_UserInterests\".\"interestVector\" <=> \"Events\".\"embedding\"), 0.0) AS \"score\"\nFROM \"Events\"\n LEFT JOIN \"_UserInterests\" ON \"_UserInterests\".\"userid\" = $2\nWHERE (\"to_tsvector\"('english', COALESCE(\"Events\".\"title\", ' ') || ' ' || COALESCE(\"Events\".\"abstract\", ' ')) @@ \"websearch_to_tsquery\"('english', CAST($1 AS TEXT)))\nORDER BY COALESCE(1 - (\"_UserInterests\".\"interestVector\" <=> \"Events\".\"embedding\"), 0.0) DESC NULLS LAST",
              "parameters" : [
                {
                  "type" : "arg",
                  "path" : "query",
                  "sqlType" : "VARCHAR"
                },
                {
                  "type" : "arg",
                  "path" : "userid",
                  "sqlType" : "VARCHAR"
                }
              ],
              "pagination" : "LIMIT_AND_OFFSET",
              "cacheDurationMs" : 0,
              "database" : "POSTGRES"
            }
          }
        },
        {
          "type" : "args",
          "parentType" : "Query",
          "fieldName" : "RecommendedEvents",
          "exec" : {
            "arguments" : [
              {
                "type" : "variable",
                "path" : "limit"
              },
              {
                "type" : "variable",
                "path" : "offset"
              },
              {
                "type" : "variable",
                "path" : "userid"
              }
            ],
            "query" : {
              "type" : "SqlQuery",
              "sql" : "SELECT \"Events\".\"url\", \"Events\".\"date\", \"Events\".\"time\", \"Events\".\"title\", \"Events\".\"abstract\", \"Events\".\"location\", \"Events\".\"speakers\", \"Events\".\"last_updated\", \"Events\".\"id\", \"Events\".\"full_text\", \"Events\".\"startTime\", \"Events\".\"embedding\", \"Events\".\"startTimestamp\", 1 - (\"_UserInterests\".\"interestVector\" <=> \"Events\".\"embedding\") AS \"score\"\nFROM \"Events\"\n INNER JOIN \"_UserInterests\" ON \"_UserInterests\".\"userid\" = $1\nORDER BY 1 - (\"_UserInterests\".\"interestVector\" <=> \"Events\".\"embedding\") DESC NULLS LAST",
              "parameters" : [
                {
                  "type" : "arg",
                  "path" : "userid",
                  "sqlType" : "VARCHAR"
                }
              ],
              "pagination" : "LIMIT_AND_OFFSET",
              "cacheDurationMs" : 0,
              "database" : "POSTGRES"
            }
          }
        }
      ],
      "mutations" : [ ],
      "subscriptions" : [ ],
      "operations" : [
        {
          "function" : {
            "name" : "GetEvents",
            "parameters" : {
              "type" : "object",
              "properties" : {
                "offset" : {
                  "type" : "integer"
                },
                "likeCount_offset" : {
                  "type" : "integer"
                },
                "limit" : {
                  "type" : "integer"
                },
                "id" : {
                  "type" : "integer"
                },
                "likeCount_limit" : {
                  "type" : "integer"
                }
              },
              "required" : [
                "id"
              ]
            }
          },
          "format" : "JSON",
          "apiQuery" : {
            "query" : "query Events($id: Long!, $limit: Int = 10, $offset: Int = 0$likeCount_limit: Int = 10, $likeCount_offset: Int = 0) {\nEvents(id: $id, limit: $limit, offset: $offset) {\nurl\ndate\ntime\ntitle\nabstract\nlocation\nspeakers {\nname\ntitle\ncompany\n}\nlast_updated\nid\nfull_text\nstartTime\nstartTimestamp\nlikeCount(limit: $likeCount_limit, offset: $likeCount_offset) {\neventid\nnum\ntest\n}\n}\n\n}",
            "queryName" : "Events",
            "operationType" : "QUERY"
          },
          "mcpMethod" : "TOOL",
          "restMethod" : "GET",
          "uriTemplate" : "queries/Events{?offset,likeCount_offset,limit,id,likeCount_limit}"
        },
        {
          "function" : {
            "name" : "GetEventsLiked",
            "parameters" : {
              "type" : "object",
              "properties" : {
                "offset" : {
                  "type" : "integer"
                },
                "likeCount_offset" : {
                  "type" : "integer"
                },
                "limit" : {
                  "type" : "integer"
                },
                "likeCount_limit" : {
                  "type" : "integer"
                },
                "userid" : {
                  "type" : "string"
                }
              },
              "required" : [
                "userid"
              ]
            }
          },
          "format" : "JSON",
          "apiQuery" : {
            "query" : "query EventsLiked($userid: String!, $limit: Int = 10, $offset: Int = 0$likeCount_limit: Int = 10, $likeCount_offset: Int = 0) {\nEventsLiked(userid: $userid, limit: $limit, offset: $offset) {\nurl\ndate\ntime\ntitle\nabstract\nlocation\nspeakers {\nname\ntitle\ncompany\n}\nlast_updated\nid\nfull_text\nstartTime\nstartTimestamp\nlikeCount(limit: $likeCount_limit, offset: $likeCount_offset) {\neventid\nnum\ntest\n}\n}\n\n}",
            "queryName" : "EventsLiked",
            "operationType" : "QUERY"
          },
          "mcpMethod" : "TOOL",
          "restMethod" : "GET",
          "uriTemplate" : "queries/EventsLiked{?offset,likeCount_offset,limit,likeCount_limit,userid}"
        },
        {
          "function" : {
            "name" : "GetPersonalizedEventSearch",
            "parameters" : {
              "type" : "object",
              "properties" : {
                "offset" : {
                  "type" : "integer"
                },
                "query" : {
                  "type" : "string"
                },
                "limit" : {
                  "type" : "integer"
                },
                "userid" : {
                  "type" : "string"
                }
              },
              "required" : [
                "query",
                "userid"
              ]
            }
          },
          "format" : "JSON",
          "apiQuery" : {
            "query" : "query PersonalizedEventSearch($query: String!, $userid: String!, $limit: Int = 10, $offset: Int = 0) {\nPersonalizedEventSearch(query: $query, userid: $userid, limit: $limit, offset: $offset) {\nurl\ndate\ntime\ntitle\nabstract\nlocation\nspeakers {\nname\ntitle\ncompany\n}\nlast_updated\nid\nfull_text\nstartTime\nstartTimestamp\nscore\n}\n\n}",
            "queryName" : "PersonalizedEventSearch",
            "operationType" : "QUERY"
          },
          "mcpMethod" : "TOOL",
          "restMethod" : "GET",
          "uriTemplate" : "queries/PersonalizedEventSearch{?offset,query,limit,userid}"
        },
        {
          "function" : {
            "name" : "GetRecommendedEvents",
            "parameters" : {
              "type" : "object",
              "properties" : {
                "offset" : {
                  "type" : "integer"
                },
                "limit" : {
                  "type" : "integer"
                },
                "userid" : {
                  "type" : "string"
                }
              },
              "required" : [
                "userid"
              ]
            }
          },
          "format" : "JSON",
          "apiQuery" : {
            "query" : "query RecommendedEvents($userid: String!, $limit: Int = 10, $offset: Int = 0) {\nRecommendedEvents(userid: $userid, limit: $limit, offset: $offset) {\nurl\ndate\ntime\ntitle\nabstract\nlocation\nspeakers {\nname\ntitle\ncompany\n}\nlast_updated\nid\nfull_text\nstartTime\nstartTimestamp\nscore\n}\n\n}",
            "queryName" : "RecommendedEvents",
            "operationType" : "QUERY"
          },
          "mcpMethod" : "TOOL",
          "restMethod" : "GET",
          "uriTemplate" : "queries/RecommendedEvents{?offset,limit,userid}"
        }
      ],
      "schema" : {
        "type" : "string",
        "schema" : "\"An RFC-3339 compliant Full Date Scalar\"\nscalar Date\n\n\"A DateTime scalar that handles both full RFC3339 and shorter timestamp formats\"\nscalar DateTime\n\ntype EventLikeCount {\n  eventid: Long!\n  num: Long!\n  test: Long!\n}\n\ntype Events {\n  url: String!\n  date: String!\n  time: String!\n  title: String!\n  abstract: String!\n  location: String!\n  speakers: [Events_speakersOutput]!\n  last_updated: DateTime!\n  id: Long!\n  full_text: String!\n  startTime: String\n  startTimestamp: DateTime\n  likeCount(limit: Int = 10, offset: Int = 0): [EventLikeCount!]\n}\n\ntype Events_speakersOutput {\n  name: String\n  title: String\n  company: String\n}\n\n\"A JSON scalar\"\nscalar JSON\n\n\"24-hour clock time value string in the format `hh:mm:ss` or `hh:mm:ss.sss`.\"\nscalar LocalTime\n\n\"A 64-bit signed integer\"\nscalar Long\n\ntype PersonalizedEventSearch {\n  url: String!\n  date: String!\n  time: String!\n  title: String!\n  abstract: String!\n  location: String!\n  speakers: [PersonalizedEventSearch_speakersOutput]!\n  last_updated: DateTime!\n  id: Long!\n  full_text: String!\n  startTime: String\n  startTimestamp: DateTime\n  score: Float!\n}\n\ntype PersonalizedEventSearch_speakersOutput {\n  name: String\n  title: String\n  company: String\n}\n\ntype Query {\n  Events(id: Long!, limit: Int = 10, offset: Int = 0): [Events!]\n  EventsLiked(userid: String!, limit: Int = 10, offset: Int = 0): [Events!]\n  PersonalizedEventSearch(query: String!, userid: String!, limit: Int = 10, offset: Int = 0): [PersonalizedEventSearch!]\n  RecommendedEvents(userid: String!, limit: Int = 10, offset: Int = 0): [RecommendedEvents!]\n}\n\ntype RecommendedEvents {\n  url: String!\n  date: String!\n  time: String!\n  title: String!\n  abstract: String!\n  location: String!\n  speakers: [RecommendedEvents_speakersOutput]!\n  last_updated: DateTime!\n  id: Long!\n  full_text: String!\n  startTime: String\n  startTimestamp: DateTime\n  score: Float\n}\n\ntype RecommendedEvents_speakersOutput {\n  name: String\n  title: String\n  company: String\n}\n\nenum _McpMethodType {\n  NONE\n  TOOL\n  RESOURCE\n}\n\nenum _RestMethodType {\n  NONE\n  GET\n  POST\n}\n\ndirective @api(mcp: _McpMethodType, rest: _RestMethodType, uri: String) on QUERY | MUTATION | FIELD_DEFINITION\n"
      }
    }
  }
}
