Re: Is there any way to perform left outer join using flink's data stream APIs

2024-08-06 Thread Kirill Ternovsky
, there does not exist any > record in B matching this field f = f1. > > In such cases I want to do a left outer join where the combined record > pushed downstream would only have field values from A and empty for > ones supposed to be joined from B. >

Re: Is there any way to perform left outer join using flink's data stream APIs

2024-08-06 Thread Sachin Mittal
t; > Kirill > > On 8/6/24 7:42 PM, Sachin Mittal wrote: > > Hi, > > I have two streams A and B. > > Which can be joined or connected using a field f. > > > > However, for a given record in A for f = f1, there does not exist any > > record in B matching th

Re: Is there any way to perform left outer join using flink's data stream APIs

2024-08-06 Thread Kirill Ternovsky
f. However, for a given record in A for f = f1, there does not exist any record in B matching this field f = f1. In such cases I want to do a left outer join where the combined record pushed downstream would only have field values from A and empty for ones supposed to be joined from B

Is there any way to perform left outer join using flink's data stream APIs

2024-08-06 Thread Sachin Mittal
Hi, I have two streams A and B. Which can be joined or connected using a field f. However, for a given record in A for f = f1, there does not exist any record in B matching this field f = f1. In such cases I want to do a left outer join where the combined record pushed downstream would only have

Apache Flink - Exception on left outer join with 'kafka' connector

2022-02-22 Thread M Singh
Hi Folks: I am using 'kafka' connector and joining with data from jdbc source (using connector).   I am using Flink v 1.14.3.  If I do a left outer join between kafka source and jdbc source, and try to save it to another kafka sink using connectors api, I get the following exception:

Re: Streaming left outer join

2016-01-28 Thread Aljoscha Krettek
with window states and triggering on time yourself. Let me > know if you need some pointers about that one. > > Cheers, > Aljoscha > > On 26 Jan 2016, at 19:32, Alexander Gryzlov wrote: > > > > Hello, > > > > I'm trying to implement a left outer join

Re: Streaming left outer join

2016-01-28 Thread Alexander Gryzlov
yourself. Let me >>> know if you need some pointers about that one. >>> >>> Cheers, >>> Aljoscha >>> > On 26 Jan 2016, at 19:32, Alexander Gryzlov >>> wrote: >>> > >>> > Hello, >>> > >>> >

Re: Streaming left outer join

2016-01-27 Thread Stephan Ewen
> know if you need some pointers about that one. >> >> Cheers, >> Aljoscha >> > On 26 Jan 2016, at 19:32, Alexander Gryzlov >> wrote: >> > >> > Hello, >> > >> > I'm trying to implement a left outer join of two Kafka streams

Re: Streaming left outer join

2016-01-27 Thread Alexander Gryzlov
me > know if you need some pointers about that one. > > Cheers, > Aljoscha > > On 26 Jan 2016, at 19:32, Alexander Gryzlov > wrote: > > > > Hello, > > > > I'm trying to implement a left outer join of two Kafka streams within a > sliding window

Re: Streaming left outer join

2016-01-27 Thread Aljoscha Krettek
pointers about that one. Cheers, Aljoscha > On 26 Jan 2016, at 19:32, Alexander Gryzlov wrote: > > Hello, > > I'm trying to implement a left outer join of two Kafka streams within a > sliding window. So far I have the following code: > > foos > .coGroup(bars)

Streaming left outer join

2016-01-26 Thread Alexander Gryzlov
Hello, I'm trying to implement a left outer join of two Kafka streams within a sliding window. So far I have the following code: foos .coGroup(bars) .where(_.baz).equalTo(_.baz) .window(SlidingTimeWindows.of(Time.of(1, TimeUnit.MINUTES), Time.of(1, TimeUnit.SECONDS))) .appl

Re: error when eun program left outer join

2015-04-27 Thread hagersaleh
I solve mu problem very thanks -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/error-when-eun-program-left-outer-join-tp1141p1146.html Sent from the Apache Flink User Mailing List archive. mailing list archive at Nabble.com.

Re: error when eun program left outer join

2015-04-27 Thread hagersaleh
implement left outer join from two dataset Customer and Orders using Tuple data type -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/error-when-eun-program-left-outer-join-tp1141p1143.html Sent from the Apache Flink User Mailing List archive

Re: error when eun program left outer join

2015-04-27 Thread Robert Metzger
Hi, what data are you using? The exception says "NullFieldException: Field 1 is null, but expected to hold a value.". Maybe the data is not in the right format? On Mon, Apr 27, 2015 at 2:32 PM, hagersaleh wrote: > I want implement left outer join from two dataset i use Tu

error when eun program left outer join

2015-04-27 Thread hagersaleh
I want implement left outer join from two dataset i use Tuple data type package org.apache.flink.examples.java.relational; import org.apache.flink.api.common.functions.CoGroupFunction; import org.apache.flink.api.common.functions.MapFunction; import org.apache.flink.api.java.DataSet; import

how can handle left outer join for any two dataset

2015-04-26 Thread hagersaleh
how can handle left outer join for any two dataset this dataset inlcude any filed number example data set one ExecutionEnvironment env = ExecutionEnvironment.getExecutionEnvironment(); DataSet> customer=env.readCsvFile("/home/hadoop/Desktop/Dataset/customer.csv") .f

Re: Left outer join

2015-04-17 Thread Fabian Hueske
There is no caching mechanism. To do the left outer join as in Tills implementation, you need to collect all elements of one! iterator in memory. If you know, that one of the two iterators contains at most 1 element, you should collect that in memory and stream the elements of the other iterator

Re: Left outer join

2015-04-17 Thread Flavio Pompermaier
Thu, Apr 16, 2015 at 10:09 PM, Flavio Pompermaier < >>>>>>> pomperma...@okkam.it> wrote: >>>>>>> >>>>>>>> I cannot find a solution to my use case :( >>>>>>>> I have 2 datasets D1 and D2 like: >>>>>>

Re: Left outer join

2015-04-17 Thread Fabian Hueske
;>>>>>> >>>>>>> D1: >>>>>>> A,p1,a1 >>>>>>> A,p2,a2 >>>>>>> A,p3,X >>>>>>> B,p3,Y >>>>>>> B,p1,b1 >>>>>>> >>>&

Re: Left outer join

2015-04-17 Thread Flavio Pompermaier
;>>> A,p3,X >>>>>> B,p3,Y >>>>>> B,p1,b1 >>>>>> >>>>>> D2: >>>>>> X,s,V >>>>>> X,r,2 >>>>>> Y,j,k >>>>>> >>>>>> I'd li

Re: Left outer join

2015-04-17 Thread Till Rohrmann
t; >>>>> I'd like to have a unique dataset D3(Tuple4) like >>>>> >>>>> A,X,a1,a2 >>>>> B,Y,b1,null >>>>> >>>>> Basically filling with >>>> D1.f1==p2)> when D1.f2==D2.f0. >>>>> Is that pos

Re: Left outer join

2015-04-17 Thread Flavio Pompermaier
a simple snippet? >>>> >>>> Thanks in advance, >>>> Flavio >>>> >>>> >>>> >>>> >>>> On Thu, Apr 16, 2015 at 9:48 PM, Till Rohrmann >>>> wrote: >>>> >>>>> You can ma

Re: Left outer join

2015-04-17 Thread Till Rohrmann
put of the right input by creating an array >>>> out of it, for example. Then you can reiterate over it. >>>> >>>> Cheers, >>>> Till >>>> On Apr 16, 2015 7:37 PM, "Flavio Pompermaier" >>>> wrote: >>>> &

Re: Left outer join

2015-04-17 Thread Flavio Pompermaier
> Hi Maximilian, >>>> I tried your solution but it doesn't work because the rightElements >>>> iterator cannot be used more than once: >>>> >>>> Caused by: org.apache.flink.util.TraversableOnceException: The Iterable >>>> can be

Re: Left outer join

2015-04-17 Thread Till Rohrmann
TraversableOnceException: The Iterable >>> can be iterated over only once. Only the first call to 'iterator()' will >>> succeed. >>> >>> On Wed, Apr 15, 2015 at 12:59 PM, Maximilian Michels >>> wrote: >>> >>>&g

Re: Left outer join

2015-04-16 Thread Flavio Pompermaier
erable >> can be iterated over only once. Only the first call to 'iterator()' will >> succeed. >> >> On Wed, Apr 15, 2015 at 12:59 PM, Maximilian Michels >> wrote: >> >>> Hi Flavio, >>> >>> Here's an simple example of

Re: Left outer join

2015-04-16 Thread Till Rohrmann
els > wrote: > >> Hi Flavio, >> >> Here's an simple example of a Left Outer Join: >> https://gist.github.com/mxm/c2e9c459a9d82c18d789 >> >> As Stephan pointed out, this can be very easily modified to construct a >> Right Outer Join (just exchange leftE

Re: Left outer join

2015-04-16 Thread Flavio Pompermaier
On Wed, Apr 15, 2015 at 12:59 PM, Maximilian Michels wrote: > Hi Flavio, > > Here's an simple example of a Left Outer Join: > https://gist.github.com/mxm/c2e9c459a9d82c18d789 > > As Stephan pointed out, this can be very easily modified to construct a > Right Outer Join (jus

Re: Left outer join

2015-04-16 Thread Maximilian Michels
ort to Tuple. > > > 2015-04-15 5:59 GMT-05:00 Maximilian Michels : > > Hi Flavio, >> >> Here's an simple example of a Left Outer Join: >> https://gist.github.com/mxm/c2e9c459a9d82c18d789 >> >> As Stephan pointed out, this can be very easil

Re: Left outer join

2015-04-16 Thread Fabian Hueske
vio, > > Here's an simple example of a Left Outer Join: > https://gist.github.com/mxm/c2e9c459a9d82c18d789 > > As Stephan pointed out, this can be very easily modified to construct a > Right Outer Join (just exchange leftElements and rightElements in the two > loops). > &

Re: Left outer join

2015-04-15 Thread Maximilian Michels
Hi Flavio, Here's an simple example of a Left Outer Join: https://gist.github.com/mxm/c2e9c459a9d82c18d789 As Stephan pointed out, this can be very easily modified to construct a Right Outer Join (just exchange leftElements and rightElements in the two loops). Here's an excerpt wit

Re: Left outer join

2015-04-15 Thread Stephan Ewen
I think this may be a great example to add as a utility function. Or actually add as an function to the DataSet, internally realized as a special case of coGroup. We do not have a ready example of that, but it should be straightforward to realize. Similar as for the join, coGroup on the join keys

Re: Left outer join

2015-04-15 Thread hagersaleh
please add link to explain left join using cogroup or add example very thanks -- View this message in context: http://apache-flink-user-mailing-list-archive.2336050.n4.nabble.com/Left-outer-join-tp1031p1034.html Sent from the Apache Flink User Mailing List archive. mailing list archive at

Re: Left outer join

2015-04-15 Thread Flavio Pompermaier
Do you have an already working example of it? :) On Wed, Apr 15, 2015 at 10:32 AM, Ufuk Celebi wrote: > > On 15 Apr 2015, at 10:30, Flavio Pompermaier wrote: > > > > > Hi to all, > > I have to join two datasets but I'd like to keep all data in the left > also if there' no right dataset. > > How

Re: Left outer join

2015-04-15 Thread Ufuk Celebi
On 15 Apr 2015, at 10:30, Flavio Pompermaier wrote: > > Hi to all, > I have to join two datasets but I'd like to keep all data in the left also if > there' no right dataset. > How can you achieve that in Flink? maybe I should use coGroup? Yes, currently you have to implement this manually wit

Left outer join

2015-04-15 Thread Flavio Pompermaier
Hi to all, I have to join two datasets but I'd like to keep all data in the left also if there' no right dataset. How can you achieve that in Flink? maybe I should use coGroup? Best, Flavio