Re: Watermarks in Event Time Temporal Join

2021-04-28 Thread Leonard Xu
Thanks for your example, Maciej I can explain more about the design. > Let's have events. > S1, id1, v1, 1 > S2, id1, v2, 1 > > Nothing is happening as none of the streams have reached the watermark. > Now let's add > S2, id2, v2, 101 > This should trigger join for id1 because we have all the kno

Re: Watermarks in Event Time Temporal Join

2021-04-28 Thread Maciej Bryński
ts. stream, id, value, ts S1, id1, v1, 1 S2, id1, v2, 1 For this events and internal join Flink will emit an event in the output stream: id1, v1, v2, 1 Despite the fact the watermark for both streams is not reached. Now similar situation for Event Time Temporal Join SELECT id, value1, value2, t

Re: Watermarks in Event Time Temporal Join

2021-04-26 Thread Leonard Xu
Hello, Maciej > I agree the watermark should pass on versioned table side, because > this is the only way to know which version of record should be used. > But if we mimics behaviour of interval join then main stream watermark > could be skipped. IIRC, rowtime interval join requires the watermark

Re: Watermarks in Event Time Temporal Join

2021-04-25 Thread Maciej Bryński
essage is late or early. If we only > use the watermark on versioned table side, we have no means to determine > whether the event in the main stream is ready to emit. > > Best, > Shengkai > > maverick 于2021年4月26日周一 上午2:31写道: >> >> Hi, >> I'm curious why

Re: Watermarks in Event Time Temporal Join

2021-04-25 Thread Shengkai Fang
curious why Event Time Temporal Join needs watermarks from both sides > to > perform join. > > Shouldn't watermark on versioned table side be enough to perform join ? > > > > > > -- > Sent from: > http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/ >

Watermarks in Event Time Temporal Join

2021-04-25 Thread maverick
Hi, I'm curious why Event Time Temporal Join needs watermarks from both sides to perform join. Shouldn't watermark on versioned table side be enough to perform join ? -- Sent from: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/

Re: Time Temporal Join

2021-03-25 Thread Satyam Shekhar
NULL > > > > I'd like to run the following query on this schema - > > > > select sum(F.C0), D.C1, tumble_start(F.R, interval '1' second) > > from F join D ON F.C1 = D.C1 > > group by D.C1, tumble(F.R, interval '1&#x

Re: Time Temporal Join

2021-03-16 Thread Timo Walther
wtime attributes must not be in the input rows of a regular join. As a workaround you can cast the time attributes of input tables to TIMESTAMP before." My understanding reading the docs is that Time Temporal Join is meant to solve this problem. So I model table D as the fol

Re: Time Temporal Join

2021-03-15 Thread Satyam Shekhar
ON F.C1 = D.C1 > group by D.C1, tumble(F.R, interval '1' second) > > However, I run into the following error while running the above query - > > "Rowtime attributes must not be in the input rows of a regular join. As a > workaround you can cast the time attrib

Time Temporal Join

2021-03-10 Thread Satyam Shekhar
und you can cast the time attributes of input tables to TIMESTAMP before." My understanding reading the docs is that Time Temporal Join is meant to solve this problem. So I model table D as the following - D: |-- C0: BIGINT |-- C1: STRING NOT NULL |-- R: TIMESTAMP(3) |-- WATERMARK

Re: Processing-time temporal join is not supported yet.

2021-03-04 Thread Leonard Xu
t...@gmail.com>> a écrit : >> Hi, Eric >> >> Firstly FileSystemTableSource doe not implement LookupTableSource which >> means we cannot directly lookup a Filesystem table. >> >> In FLINK-19830, we plan to support Processing-time temporal join any >> table

Re: Processing-time temporal join is not supported yet.

2021-03-04 Thread eric hoffmann
eonard Xu a écrit : > >> Hi, Eric >> >> Firstly FileSystemTableSource doe not implement LookupTableSource which >> means we cannot directly lookup a Filesystem table. >> >> In FLINK-19830, we plan to support Processing-time temporal join any >> table/views by lookup

Re: Processing-time temporal join is not supported yet.

2021-03-03 Thread Leonard Xu
t directly lookup a Filesystem table. > > In FLINK-19830, we plan to support Processing-time temporal join any > table/views by lookup the data in join operator state which scanned from the > filesystem table, but as the issue described: join processing for left stream > doesn'

Re: Processing-time temporal join is not supported yet.

2021-03-03 Thread eric hoffmann
écrit : > Hi, Eric > > Firstly FileSystemTableSource doe not implement LookupTableSource which > means we cannot directly lookup a Filesystem table. > > In FLINK-19830, we plan to support Processing-time temporal join any > table/views by lookup the data in join operator sta

Re: Processing-time temporal join is not supported yet.

2021-02-26 Thread Leonard Xu
Hi, Eric Firstly FileSystemTableSource doe not implement LookupTableSource which means we cannot directly lookup a Filesystem table. In FLINK-19830, we plan to support Processing-time temporal join any table/views by lookup the data in join operator state which scanned from the filesystem

Re: Processing-time temporal join is not supported yet.

2021-02-26 Thread Matthias Pohl
nk 1.12.1 i read in the doc that Processing-time temporal > join is supported for kv like join but when i try i get a: > > Exception in thread "main" org.apache.flink.table.api.TableException: > Processing-time temporal join i

Processing-time temporal join is not supported yet.

2021-02-26 Thread eric hoffmann
Hello Working with flink 1.12.1 i read in the doc that Processing-time temporal join is supported for kv like join but when i try i get a: Exception in thread "main" org.apache.flink.table.api.TableException: Processing-time temporal join is not supported yet.