Re: Regarding Project proposal review and feedback

2023-04-29 Thread David Morávek
I. > And thanks for the feedback. > > Best Regards, > Siddharth Aryan > > On Thu, Apr 27, 2023 at 1:39 PM Jeff Zhang wrote: > >> Same question as David, one idea in my mind is to integrate the beam sql >> api with flink table api, this does not exist in the curre

Re: Regarding Project proposal review and feedback

2023-04-27 Thread David Morávek
Hi Siddharth, Thanks for your interest in the Flink Runner for Beam. Reading through the project, one thing that immediately strikes me is that there already is a Flink runner based on DataStream and Operator (one level below DataStream) API in the code base. Are you aware of this? If yes, how

Re: Flink support for OrderedListState

2021-12-22 Thread David Morávek
instead of timestamp. Timestamp is likely the more-common >> use case, but I can imagine other use cases (e.g. imagining reading log >> entries and wanting to keep the entries sorted by LSN). >> >> On Mon, Nov 22, 2021 at 1:12 PM David Morávek wrote: >> &

Re: Flink support for OrderedListState

2021-11-22 Thread David Morávek
community. Would you be willing to participate in this contribution? [1] https://github.com/dmvk/flink/commit/ecdbc774b13b515e8c0943b2c143fb1e34eca6f0 Best, D. On Tue, Nov 16, 2021 at 8:34 PM David Morávek wrote: > Yes, my intuition is that it should be more or less the same. There might > b

Re: Flink support for OrderedListState

2021-11-16 Thread David Morávek
in memory after reading them, but in this particular case it should save some cycles. On Tue, Nov 16, 2021 at 8:00 PM Reuven Lax wrote: > Right, but no worse than storing in ListState for that case, right? > > On Tue, Nov 16, 2021 at 10:44 AM David Morávek wrote: > >> Based on

Re: Flink support for OrderedListState

2021-11-16 Thread David Morávek
> > Based on what Aljoscha said, storing it in MapState would be better as in > practice the map will be sorted (as long as the key encoding is order > preserving, and on quick glance InstantCoder is order preserving), so I'll > change that This property only holds for RocksDB based state

Re: Flink support for OrderedListState

2021-11-16 Thread David Morávek
Hi Reuven, this would be a great addition to the Flink Runner and could help with broader adoption ;) to make an effective implementation that works well across different state backends, this will most likely require adding a new primitive state type to the Flink's state backend ecosystem. I'll

Re: Beam with Flink runner - Issues when writing to S3 in Parquet Format

2021-09-14 Thread David Morávek
Hi Sandeep, Jan has already provided pretty good guidelines for getting more context on the issue ;) Because this is not for the first time, I would like to raise awareness, that it's not OK to send a user related question to four Apache mailing list (that I know of). Namely: -

Re: Unable to read state Witten by Beam application with Flink runner using Flink's State Processor API

2021-08-06 Thread David Morávek
David Morávek wrote: > Hi Sandeep, thanks for the example, I'll take a look into it and will get > back to you ;) > > On Tue, Aug 3, 2021 at 9:44 PM Kathula, Sandeep < > sandeep_kath...@intuit.com> wrote: > >> Hi David, >> Thanks for the rep

Re: [DISCUSS] Drop support for Flink 1.10

2021-05-31 Thread David Morávek
Hi, +1 as we've agreed to keep support for three latest major releases in the past D. On Mon, May 31, 2021 at 9:54 AM Jan Lukavský wrote: > Hi, > > +1 to remove the support for 1.10. > > Jan > On 5/28/21 10:00 PM, Ismaël Mejía wrote: > > Hello, > > With Beam support for Flink 1.13 just

Re: [DISCUSS] Drop support for Flink 1.8 and 1.9

2021-03-12 Thread David Morávek
+1 D. On Thu, Mar 11, 2021 at 8:33 PM Ismaël Mejía wrote: > +user > > > Should we add a warning or something to 2.29.0? > > Sounds like a good idea. > > > > > On Thu, Mar 11, 2021 at 7:24 PM Kenneth Knowles wrote: > > > > Should we add a warning or something to 2.29.0? > > > > On Thu, Mar 11,

Re: Possible 80% reduction in overhead for flink runner, input needed

2020-10-27 Thread David Morávek
of it, can you link me to some discussions about this in the past? > > > >Thanks for reply :D > > > >Best regards, > >Teodor Spæren > > > >[1]: > https://beam.apache.org/documentation/programming-guide/#immutability > > > > > >On Tue, Oct 27, 2020

Re: Possible 80% reduction in overhead for flink runner, input needed

2020-10-27 Thread David Morávek
Hi Teodor, Thanks for bringing this up. This is a known, long standing "issue". Unfortunately there are few things we need to consider: - As you correctly noted, the *Beam model doesn't enforce immutability* of input / output elements, so this is the price. - We* can not break *existing

Re: Remove EOL'd Runners

2020-06-10 Thread David Morávek
+1 On Tue, Jun 9, 2020 at 7:43 PM Ahmet Altay wrote: > Thank you Tyson! > > On Tue, Jun 9, 2020 at 10:20 AM Thomas Weise wrote: > >> +1 >> >> >> On Tue, Jun 9, 2020 at 9:41 AM Robert Bradshaw >> wrote: >> >>> Makes sense to me. >>> >>> On Tue, Jun 9, 2020 at 8:45 AM Maximilian Michels >>>

dealing with late data output timestamps

2020-05-28 Thread David Morávek
Hi, I've came across "unexpected" model behaviour when dealing with late data and custom timestamp combiners. Let's take a following pipeline as an example: final PCollection input = ...; input.apply( "GlobalWindows", Window.into(new GlobalWindows()) .triggering(

snapshot releases

2020-05-13 Thread David Morávek
Hi, I've just lost quite some time on the issue related to how we currently release beam snapshots. Because we only publish snapshots from the master branch, unreleased version of beam (2.21.0-SNAPSHOT) was last updated on 9th of April. eg.

Re: Validates Runner on Java 11 and the Java SDK Harness

2020-05-07 Thread David Morávek
Great effort Ismaël! ;) Can't wait to try this out :) On Thu, May 7, 2020 at 12:08 PM Ismaël Mejía wrote: > Filled https://issues.apache.org/jira/browse/BEAM-9915 for the moment > to track this. > > On Wed, Apr 22, 2020 at 10:35 PM Mikhail Gryzykhin > wrote: > > > > +Paweł Pasterz > > > > On

Re: Flink: Lost pane timing at some steps of pipeline

2020-05-04 Thread David Morávek
Hi Jozef, I think this is expected beahior as Flink does not use default expansion for Reshuffle (uses round-robin rebalance ship strategy instead). There is no aggregation that needs buffering (and triggering). All of the elements are immediately emmited to downstream operations after the

Re: Flink Runner with RequiresStableInput fails after a certain number of checkpoints

2020-04-21 Thread David Morávek
Hi Stephen, nice catch and awesome report! ;) This definitely needs a proper fix. I've created a new JIRA to track the issue and will try to resolve it soon as this seems critical to me. https://issues.apache.org/jira/browse/BEAM-9794 Thanks, D. On Mon, Apr 20, 2020 at 10:41 PM Stephen Patel

Re: [VOTE] Accept the Firefly design donation as Beam Mascot - Deadline Mon April 6

2020-04-06 Thread David Morávek
+1 (non-binding) On Mon, Apr 6, 2020 at 12:51 PM Reza Rokni wrote: > +1(non-binding) > > On Mon, Apr 6, 2020 at 5:24 PM Alexey Romanenko > wrote: > >> +1 (non-binding). >> >> > On 3 Apr 2020, at 14:53, Maximilian Michels wrote: >> > >> > +1 (binding) >> > >> > On 03.04.20 10:33, Jan Lukavský

Re: [DISCUSS] Drop support for Flink 1.7

2020-02-19 Thread David Morávek
+1 for dropping 1.7, once we have 1.10 support ready D. On Tue, Feb 18, 2020 at 7:01 PM wrote: > Hi Ismael, > yes, sure. The proposal would be to have snapshot dependency in the > feature branch. The snapshot must be changed to release before merge to > master. > Jan > > Dne 18. 2. 2020 17:55

Re: [DISCUSS] BIP reloaded

2019-12-09 Thread David Morávek
Hi Jan, I think this is more pretty much what we currently do, just a little bit more transparent for the community. If the process is standardized, it can open doors for bigger contributions from people not familiar with the process. Also it's way easier to track progress of BIPs, than documents

Re: Update on push-down for SQL IOs.

2019-11-28 Thread David Morávek
Nice, this should bring a great performance improvement for SQL. Thanks for your work! On Thu, Nov 28, 2019 at 6:33 AM Kenneth Knowles wrote: > Nice! Thanks for the very thorough summary. I think this will be a really > good thing for Beam. Most of the IO sources are very highly optimized for >

Re: Full stream-stream join semantics

2019-11-28 Thread David Morávek
that. It is the sort of thing that already exists in some engines so >> there's proof of concept :-). Jan makes the good point that executing the >> same join in batch you wouldn't use the same algorithm, because the >> disorder will be unbounded. In Beam you'd want a PTransform that expands >

Re: Full stream-stream join semantics

2019-11-26 Thread David Morávek
ortedDoFn, because > of the state annotation definitions) > > Jan > On 11/26/19 12:56 PM, David Morávek wrote: > > Hi, > > I think what Jan has in mind would look something like this > <https://gist.github.com/dmvk/3ea32eb36c6406fa72d70b9b1df1d878>, if > implemented in

Re: Full stream-stream join semantics

2019-11-26 Thread David Morávek
Hi, I think what Jan has in mind would look something like this , if implemented in user code. Am I right? D. On Tue, Nov 26, 2019 at 10:23 AM Jan Lukavský wrote: > > On 11/25/19 11:45 PM, Kenneth Knowles wrote: > > > > On Mon,

Re: ES 7.0 Support Development

2019-11-07 Thread David Morávek
Hi Zhong, just fyi, there is another ongoing effort on adding es 7 support. https://github.com/apache/beam/pull/10025 you guys should get in touch ;) D. Sent from my iPhone > On 7 Nov 2019, at 20:20, Zhong Chen wrote: > >  > Hi all, > > I have made a PR for adding ES 7.0 support here.

Re: Multiple iterations after GroupByKey with SparkRunner

2019-09-27 Thread David Morávek
Hi, Spark's GBK is currently implemented using `sortBy(key and value).mapPartition(...)` for non-merging windowing in order to support large keys and large scale shuffles. Merging windowing is implemented using standard GBK (underlying spark impl. uses ListCombiner + Hash Grouping), which is by

Re: [DISCUSS] Supporting multiple Flink versions vs. tech debt

2019-09-09 Thread David Morávek
SerializerFactory(), new >> > > /v1_5_to_1_7./EncodedValueTypeSerializerFactory())/ >> > > + *1.8/* >> > > - build.gradle >> > > + *src/* >> > >- org/apache/beam/runners/flink/*FlinkRunner.java */# >> > > FlinkRun

[DISCUSS] Supporting multiple Flink versions vs. tech debt

2019-09-07 Thread David Morávek
Hello, we currently have an opened PR for Flink 1.9 , which greatly improves the runner for batch use-case. In case the PR gets merged, we would be supporting 5 latest major versions of Flink, which obviously come with high maintenance price and makes

Re: Cassandra flaky on Jenkins?

2019-09-04 Thread David Morávek
8025 > > > > Would be good if you could take a look JB. > > > > Thanks, > > Max > > > > On 03.09.19 15:32, David Morávek wrote: > >> yes, that looks similar. example: > >> > >> https://github.com/apache/beam/pull/9464 > >

Re: Cassandra flaky on Jenkins?

2019-09-03 Thread David Morávek
jira/browse/BEAM-7355 right ? > > Regards > JB > >> On 03/09/2019 15:11, David Morávek wrote: >> I’m running into these failures too >> >> D. >> >> Sent from my iPhone >> >>> On 3 Sep 2019, at 14:34, Jean-Baptiste Onofré wrote

Re: Cassandra flaky on Jenkins?

2019-09-03 Thread David Morávek
I’m running into these failures too D. Sent from my iPhone > On 3 Sep 2019, at 14:34, Jean-Baptiste Onofré wrote: > > Hi, > > Let me take a look. Do you always have this issue on Jenkins or randomly ? > > Regards > JB > >> On 03/09/2019 14:19, Alex Van Boxel wrote: >> Hi, is it only me

Re: [ANNOUNCE] New committer: Kyle Weaver

2019-08-06 Thread David Morávek
Congratulations Kyle!! Sent from my iPhone > On 6 Aug 2019, at 18:47, Anton Kedin wrote: > > Congrats! > >> On Tue, Aug 6, 2019, 9:37 AM Ankur Goenka wrote: >> Congratulations Kyle! >> >>> On Tue, Aug 6, 2019 at 9:35 AM Ahmet Altay wrote: >>> Hi, >>> >>> Please join me and the rest of the

Re: Support ZetaSQL as a new SQL dialect in BeamSQL

2019-08-04 Thread David Morávek
Hi Rui, This is definitely an interesting topic! Can you please elaborate little bit more about the benefits, that this will bring to the end user? All the documents only cover technical details and I'm still not sure what you're trying to achieve product-wise. Best, D. On Sun, Aug 4, 2019 at

Re: [ANNOUNCE] New committer: Robert Burke

2019-08-01 Thread David Morávek
Congratulations Robert! On Thu, Aug 1, 2019 at 8:44 PM Maximilian Michels wrote: > Go Robert! ;) Congrats > > On 30.07.19 19:38, Mark Liu wrote: > > Congratulations Robert! > > > > On Thu, Jul 18, 2019 at 9:49 AM 송원욱 > > wrote: > > > > Congrats Robert! > > > > > >

Re: [ANNOUNCE] New committer: Jan Lukavský

2019-07-31 Thread David Morávek
Congratulations Jan, well deserved! ;) D. On Wed, Jul 31, 2019 at 10:17 AM Ryan Skraba wrote: > Congratulations Jan! > > On Wed, Jul 31, 2019 at 10:10 AM Ismaël Mejía wrote: > > > > Hi, > > > > Please join me and the rest of the Beam PMC in welcoming a new > > committer: Jan Lukavský. > > > >

Re: [VOTE] Vendored dependencies release process

2019-07-06 Thread David Morávek
+1 Sent from my iPhone > On 6 Jul 2019, at 11:25, Lukasz Cwik wrote: > > +1 > >> On Wed, Jul 3, 2019 at 10:24 AM Jens Nyman wrote: >> +1 >> >> On 2019/07/02 23:49:10, Lukasz Cwik wrote: >> > Please vote based on the vendored dependencies release process as> >> > discussed[1] and

Re: kafka 0.9 support

2019-04-03 Thread David Morávek
nnett wrote: > I withdraw my concern -- checked on info on the cluster I will eventually > access. It is on 0.8, so I was speaking too soon. Can't speak to rest of > user base. > > On Tue, Apr 2, 2019 at 11:03 AM Raghu Angadi wrote: > >> Thanks to David Morávek for pointing

kafka 0.9 support

2019-04-01 Thread David Morávek
Hello, is there still a reason to keep Kafka 0.9 support? This unfortunately adds lot of complexity to KafkaIO implementation. Kafka 0.9 was released on Nov 2015. My first shot on removing Kafka 0.9 support would remove second consumer, which is used for fetching offsets. WDYT? Is this support

docs: java-dependencies

2019-03-24 Thread David Morávek
Hello, I've run into java-dependencies page , while upgrading joda-time. This page is supposed to track dependencies across releases, but the last entry is for 2.9 release. Do we still want to maintain this? If so, could we somehow

Re: joda-time dependency version

2019-03-23 Thread David Morávek
MM-dd HH:mm:ss" >> format, to "hh:mm:ss yy-MMM-dd z" format, the converted value is "01:56:12 >> 19-Mar-15 PDT". >> > > >> > > The javadoc for both the versions doesn't seem different though, for >> 'z' DateTimeFormat. >> >

Re: [Announcement] New Website for Beam Summits

2019-03-20 Thread David Morávek
This is great! Thanks for all of the hard work you're putting into this. D. On Wed, Mar 20, 2019 at 1:38 PM Maximilian Michels wrote: > Not a bug, it's a feature ;) > > On 20.03.19 07:23, Kenneth Knowles wrote: > > Very nice. I appreciate the emphasis on coffee [1] [2] [3] though I > > suspect

Re: [ANNOUNCE] New committer announcement: Michael Luckey

2019-02-27 Thread David Morávek
Congrats Michael!  D. > On 28 Feb 2019, at 03:27, Ismaël Mejía wrote: > > Congratulations Michael, and thanks for all the contributions! > >> On Wed, Feb 27, 2019 at 6:30 PM Ankur Goenka wrote: >> >> Congratulations Michael! >> >>> On Wed, Feb 27, 2019 at 2:25 PM Thomas Weise wrote: >>>

Re: [ANNOUNCE] New committer announcement: Gleb Kanterov

2019-01-25 Thread David Morávek
Congratulations! Sent from my iPhone > On 25 Jan 2019, at 20:41, Kai Jiang wrote: > > Congratulations! > >> On Fri, Jan 25, 2019 at 10:01 AM Rui Wang wrote: >> Congratulations! >> >> -Rui >> >>> On Fri, Jan 25, 2019 at 9:58 AM Ruoyun Huang wrote: >>> Congratulations Gleb! >>> On

Re: Spark-optimized Shuffle (SOS) any update?

2018-12-20 Thread David Morávek
of tasks low. Am I missing something? Thanks, D. On Thu, Dec 20, 2018 at 12:27 PM David Morávek wrote: > This is an awesome news! Is there anything we can do to help? We are > currently facing huge performance penalties due this issue. > > Thanks, > David > > On Wed, Dec 19,

Re: Spark-optimized Shuffle (SOS) any update?

2018-12-20 Thread David Morávek
This is an awesome news! Is there anything we can do to help? We are currently facing huge performance penalties due this issue. Thanks, David On Wed, Dec 19, 2018 at 5:43 PM Ilan Filonenko wrote: > Recently, the community has actively been working on this. The JIRA to > follow is: >

Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-12-05 Thread David Morávek
a blocker ? If not, I think we should hold it till 2.10 > (approx. 4 - 6 weeks). If you think it's a blocker I can build the candidate > again and run tests. > > Thanks, > Cham > >> On Wed, Dec 5, 2018 at 1:37 AM David Morávek wrote: >> Would it be possible to cherry

Re: [PROPOSAL] Prepare Beam 2.9.0 release

2018-12-05 Thread David Morávek
Would it be possible to cherry pick this patch into 2.9? It removes forgotten kryo dependency from Euphoria DSL, which we don't want user to start depending on, unless *beam-sdks-java-extension-kryo* dependency is explicitly used. https://github.com/apache/beam/pull/7195 Thanks, D. On Tue, Dec

Re: [DISCUSS] Structuring Java based DSLs

2018-11-30 Thread David Morávek
Jan, we made Kryo optional recently (it is a separate module and is used only in tests). From a quick look it seems that we forgot to remove compile time dependency from euphoria's *build.gradle*. Only "strong" dependencies I'm aware of are core SDK and guava. We'll be probably adding sketching

[Euphoria] Looking for a reviewer.

2018-11-06 Thread David Morávek
Hello, I'm looking for a reviewer for [BEAM-5790] and also for any upcoming Euphoria PR which I submit. It has been already reviewed internally, but it should be also reviewed by a committer who did not author the code. It would be also great if other

Re: Growing Beam -- A call for ideas? What is missing? What would be good to see?

2018-10-27 Thread David Morávek
Hello Alejandro, +1 for java implementation, even though this would probably require more effort from your side The main problem with Scio is that it lives outside beam code base and depends on specific version of Beam SDK. The sketching extension (and any other module in beam code base) on the

Re: [DISCUSS] Publish vendored dependencies independently

2018-10-23 Thread David Morávek
+1 This should improve build times a lot. It would be great if vendored deps could stay in the main repository. D. On Tue, Oct 23, 2018 at 12:21 PM Maximilian Michels wrote: > Looks great, Kenn! > > > Max: what is the story behind having a separate flink-shaded repo? Did > that make it easier

Re: [PROPOSAL] Move sorting to sdks-java-core

2018-10-22 Thread David Morávek
Mon, Oct 22, 2018 at 8:52 AM David Morávek > wrote: > >> What should be the next step? I guess we all agree that hadoop dependency >> should be splitted out. Then we're left off with the SortValues transform + >> in memory implementation. I'm ok with keeping this as

Re: [PROPOSAL] Move sorting to sdks-java-core

2018-10-22 Thread David Morávek
SequenceFile > <http://hadoop.apache.org/docs/stable/api/index.html?org/apache/hadoop/io/SequenceFile.html> > for > on-disk sorting. > > > > On Thu, 18 Oct 2018 at 11:19 David Morávek > wrote: > >> Kenn, I believe we should not introduce hadoop dependency to neither s

Re: [DISCUSS] Beam public roadmap

2018-10-20 Thread David Morávek
+1 this looks like a great starting point. It is always beneficial for the user to know where the project is headed. Sent from my iPhone > On 20 Oct 2018, at 03:09, Ahmet Altay wrote: > > I looked at #6718, I think this is great as a starting point and not just a > mock. I particularly like

Re: [PROPOSAL] Move sorting to sdks-java-core

2018-10-18 Thread David Morávek
ipelines >>> executed via portability since the Runner will be able to perform >>> PTransform replacement and optimization based upon the URN of the transform >>> and its payload so it would never need to have the "Sorter" class in its >>> classpath. >>>

Re: [PROPOSAL] Move sorting to sdks-java-core

2018-10-17 Thread David Morávek
17, 2018 at 11:10 PM Reuven Lax wrote: > One concern would be merging windows. This happens after shuffle, so even > if the shuffle were sorted you would need to do a sorted merge of two > sorted buffers. > > On Wed, Oct 17, 2018 at 2:08 PM David Morávek > wrote: > &

[PROPOSAL] Move sorting to sdks-java-core

2018-10-17 Thread David Morávek
Hello, I want to summarize my thoughts on the per key value sorting. Currently we have a separate module for sorting extension. The extension contains *SortValues* transformation and implementations of different sorters. Performance-wise it would be great to be able* to delegate sorting to a

Re: [Proposal] Euphoria DSL - looking for reviewers

2018-10-17 Thread David Morávek
1, 2018 at 10:06 AM Kenneth Knowles wrote: >> >>> I've filed the IP Clearance. I'll report back here. >>> >>> Kenn >>> >>> On Wed, Oct 10, 2018 at 3:33 PM David Morávek >>> wrote: >>> >>>> >>>> >>&

Re: [Call for items] October Beam Newsletter

2018-10-17 Thread David Morávek
Thomas thanks for the reminder! I've added following: - Euphoria DSL - HadoopFormatIO - Beam meetup Prague, Czech Republic On Wed, Oct 17, 2018 at 4:32 PM Etienne Chauchot wrote: > Hi Rose, > Thanks for the reminder, I added > - ongoing RabbitMQ IO > - and already done Graphite metrics sink. >

Re: [Proposal] Euphoria DSL - looking for reviewers

2018-10-14 Thread David Morávek
Thanks Kenn and Reuven! This brings up the question, how should we proceed with the further development? Up until now, we did all changes in our own repository, which was very flexible as we could do code reviews and PR merges by ourselves. We would love to take a full responsibility for the

Re: [Proposal] Euphoria DSL - looking for reviewers

2018-10-10 Thread David Morávek
SDK, >> if I didn't miss anything; >> - it has people working on it and supporting it; >> >> All other issues can probably be sorted out in normal Beam process. >> >> Regards, >> Anton >> >> On Wed, Oct 10, 2018 at 5:57 AM David Morávek >

Re: [Proposal] Euphoria DSL - looking for reviewers

2018-10-10 Thread David Morávek
Hello Max, It would be great if you can do more of a "general" review, the code base is fairly large, well tested and it was already reviewed internally by several people. We would like to have the overall approach and design decisions validated by the community and get some inputs on what could

Re: SparkRunner - GroupByKey

2018-09-17 Thread David Morávek
Thanks! I've created BEAM-5392 <https://issues.apache.org/jira/browse/BEAM-5392> to track the issue. On Fri, Sep 14, 2018 at 4:46 PM Robert Bradshaw wrote: > On Fri, Sep 14, 2018 at 4:22 PM David Morávek > wrote: > >> Hello Robert, >> >> thanks for th

Re: [VOTE] Donating the Dataflow Worker code to Apache Beam

2018-09-14 Thread David Morávek
+1 > On 15 Sep 2018, at 00:59, Anton Kedin wrote: > > +1 > >> On Fri, Sep 14, 2018 at 3:22 PM Alan Myrvold wrote: >> +1 >> >>> On Fri, Sep 14, 2018 at 3:16 PM Boyuan Zhang wrote: >>> +1 >>> On Fri, Sep 14, 2018 at 3:15 PM Henning Rohde wrote: +1 > On Fri, Sep 14,

Re: SparkRunner - GroupByKey

2018-09-14 Thread David Morávek
e key resulting in smaller groupings. I'm not sure I > understand your output requirements enough to know if this would work. > > On Fri, Sep 14, 2018 at 3:28 PM David Morávek > wrote: > >> Hello, >> >> currently, we are trying to move one of our large scale batch jo

SparkRunner - GroupByKey

2018-09-14 Thread David Morávek
Hello, currently, we are trying to move one of our large scale batch jobs (~100TB inputs) from our Euphoria based SparkRunner to Beam's Spark runner and we came across the following issue. Because we rely on hadoop ecosystem, we need to group outputs by

Re: [PROPOSAL] (BEAM-5309) Support for streaming HadoopOutputFormatIO

2018-09-13 Thread David Morávek
sorry, document link is here: https://s.apache.org/beam-streaming-hofio :) On Thu, Sep 13, 2018 at 4:47 PM David Morávek wrote: > Hello Beamers, > > my team is currently starting to work on streaming support for the > HadoopOutputFormatIO introduced by Alexey. I've tried to

[PROPOSAL] (BEAM-5309) Support for streaming HadoopOutputFormatIO

2018-09-13 Thread David Morávek
Hello Beamers, my team is currently starting to work on streaming support for the HadoopOutputFormatIO introduced by Alexey. I've tried to summarize my thoughts about the implementation details in the short design document. It would be great, if the document can get a community review as there

Re: [DISCUSS] Unification of Hadoop related IO modules

2018-09-07 Thread David Morávek
+1 for option 3 as it should be the least painful option for the current users D. Sent from my iPhone > On 7 Sep 2018, at 19:50, Tim wrote: > > Another +1 for option 3 (and preference of HadoopFormatIO naming). > > Thanks Alexey, > > Tim > > >> On 7 Sep 2018, at 19:13, Andrew Pilloud

Re: Survey: what is everyone working on that you want to share?

2018-05-28 Thread David Morávek
>> >> I sent https://github.com/apache/beam-site/pull/441 to cover efforts >> related to Python 3 support in Beam. >> >> Thanks, >> Valentyn >> >> On Tue, May 15, 2018 at 10:27 AM, David Morávek <david.mora...@gmail.com> >> wrote: >>

Re: Survey: what is everyone working on that you want to share?

2018-05-15 Thread David Morávek
Hi Kenn, Java 8 DSL JIRA: dsl-euphoria / BEAM-3900

Re: Euphoria Java 8 DSL - proposal

2018-02-27 Thread David Morávek
of this > donation? > > > > On Tue, Jan 2, 2018 at 10:52 PM, Jean-Baptiste Onofré <j...@nanthrax.net > > <mailto:j...@nanthrax.net>> wrote: > > > > Great ! > > > > Thanks ! > > Regards > > JB > > > >

Re: Euphoria Java 8 DSL - proposal

2018-01-02 Thread David Morávek
> Are you on Slack and/or hangout to plan this ? > > Thanks, > Regards > JB > > On 01/02/2018 05:35 PM, David Morávek wrote: > >> Hello JB, >> >> can we help in any way to move things forward? >> >> Thanks, >> D. >> >> On Mon, Dec

Re: Euphoria Java 8 DSL - proposal

2018-01-02 Thread David Morávek
gt;>>>> am curious about what parts of the Euphoria model you guys had to >>>>> sacrifice to support Beam, and what parts of Beam's model should still >>>>> be integrated into Euphoria (and if there is a straightforward path to >>>>> do it). >>

Re: Euphoria Java 8 DSL - proposal

2017-12-18 Thread David Morávek
ke a look on your wordcount examples to give you a >>> complete feedback. I like the idea and a fluent Java DSL is valuable. >>> >>> Let's wait feedback from others. If we have a consensus, then I would be >>> more than happy to help you for the donation

Euphoria Java 8 DSL - proposal

2017-12-17 Thread David Morávek
Hello, First of all, thanks for the amazing work the Apache Beam community is doing! In 2014, we've started development of the runtime independent Java 8 API, that helps us to create unified big-data processing flows. It has been used as a core building block of Seznam.cz web crawler data