Re: [Proposal] | Move FileIO and TextIO from :sdks:java:core to :sdks:java:io:file

2022-12-15 Thread Cristian Constantinescu
Counter argument to the "in one box" thing. I would like to point out that "having things in one box" is not a reason to have the code residing in the same module/project. What the user sees and how the code is structured are two very different things in my opinion. Beam can certainly have

Re: [Proposal] | Move FileIO and TextIO from :sdks:java:core to :sdks:java:io:file

2022-12-12 Thread Cristian Constantinescu
Hi, "As for the pipeline update feature, we've long discussed having "pick-your-implementation" transforms that specify alternative, equivalent implementations." Could someone point me to where this was discussed please? I seem to have missed that whole topic. Is it like a dependency injection

Re: [DISCUSSION][JAVA] Current state of Java 17 support

2022-12-01 Thread Cristian Constantinescu
Hi, I came across some Kafka info and would like to share for those unaware. Kafka is planning to drop support for Java 8 in Kafka 4 (Java 8 is deprecated in Kafka 3), see KIP-750 [1]. I'm not sure when Kafka 4 is scheduled to be released (probably a few years down the road), but when it

Re: Using unbounded source as a side input for a DoFn

2022-07-21 Thread Cristian Constantinescu
Disclaimer: I am not an expert, but I kinda worked on something similar. A few points I'd like to bring up: - Side inputs do not trigger the processElement function when new elements are added to the input. That means that if your side input doesn't have the desired other item in the side input

Re: [DISCUSS] Next steps for update of Avro dependency in Beam

2022-05-13 Thread Cristian Constantinescu
Hey everyone, I appreciate that the Beam community is having another look at this. I don't have any strong opinions on these options, but think that extracting Avro in its own extension is the better choice. I've already given a stab at extracting Avro in its own module (PR: 12748 [1]). The way

Re: Bean 2.36.0 + Flink 1.13 appears to be broken

2022-02-08 Thread Cristian Constantinescu
; I often use "getProtectionDomain()" > https://stackoverflow.com/a/56000383/975074 to find the JAR file from a > class. > > > On Tue, Feb 8, 2022 at 3:18 PM Cristian Constantinescu > wrote: > >> Hi everyone, >> >> I am very excited with the 2.36 release, espe

Bean 2.36.0 + Flink 1.13 appears to be broken

2022-02-08 Thread Cristian Constantinescu
Hi everyone, I am very excited with the 2.36 release, especially the stopReadOffset addition to the KafkaSourceDescriptors. With it, I can read sections of a topic and create state,effectively having a bounded kafka source, before reading new items that need processing. Unfortunately, running

Re: Beam State with the Flink Runner when things go wrong

2022-02-04 Thread Cristian Constantinescu
programmatically before Pipeline.create(options) call is made. Cheers, Cristian [1] https://beam.apache.org/documentation/runners/flink/ On Fri, Feb 4, 2022 at 10:17 AM Cristian Constantinescu wrote: > Hey Jan, > > I agree that silently ignoring the parameter is misleading and, in

Re: Beam State with the Flink Runner when things go wrong

2022-02-04 Thread Cristian Constantinescu
gt; > Would you file a JIRA? Or possibly create a PR to fix this? > > Best, > > Jan > On 2/3/22 07:12, Cristian Constantinescu wrote: > > Hi everyone, > > I've done some digging within the Beam source code. It looks like when the > flinkMaster argument is not set, the sav

Re: [DISCUSS] Migrate Jira to GitHub Issues?

2022-01-31 Thread Cristian Constantinescu
I've been semi-following this thread, apologies if this has been raised already. >From a user point of view, in some corporate environments (that I've worked at), Github is blocked. That includes the issues part. The Apache Jira is not blocked and does at times provide value while investigating

Potential bug: AutoValue + Memoized fields

2021-10-28 Thread Cristian Constantinescu
Hi everyone, Looks like Beam has a little bit of an issue when using AutoValues with Memoized (cached) fields. It's not a big issue, and the workaround is simply not using Memoised fields at the cost of a little performance. (See comment in code snippet) The code further below produces this

Re: Potential Bug: Beam + Flink + AutoValue Builders

2021-10-26 Thread Cristian Constantinescu
ncy classloader munging, and that might be breaking an assumption in this > code. Passing in the correct classloader should hopefully fix this. > > Reuven > > > On Tue, Oct 26, 2021 at 10:59 AM Cristian Constantinescu > wrote: > >> Hi everyone, >> >&g

Potential Bug: Beam + Flink + AutoValue Builders

2021-10-26 Thread Cristian Constantinescu
Hi everyone, Not sure if anyone is using Beam with the Flink Runner and AutoValue builders. For me, it doesn't work. I have some questions and a workaround for anyone in the same boat. Beam 2.31, Flink 1.13, AutoValue 1.8.2 Here's the code: package org.whatever.testing; import

Re: Why is Avro Date field using InstantCoder?

2021-10-22 Thread Cristian Constantinescu
to the beam >> repo will take 1-2 months to make it to a non-snapshot build even if you do >> find a long term solution acceptable to all interested parties. >> >> -Daniel >> >> On Mon, Oct 18, 2021 at 1:46 PM Cristian Constantinescu >> wrote: >> >>>

Re: Why is Avro Date field using InstantCoder?

2021-10-17 Thread Cristian Constantinescu
ion code at runtime with > ByteBuddy, we could potentially just generate different conversions > depending on the Avro version. > > On Fri, Oct 15, 2021 at 11:56 PM Cristian Constantinescu > wrote: > >> Those are fair points. However please consider that there might be n

Re: Why is Avro Date field using InstantCoder?

2021-10-16 Thread Cristian Constantinescu
guess what I'm saying is that there's definitely a non-negligible cost associated with old 3rd party libs in Beam's code (even if efforts are put in to minimize them). On Sat, Oct 16, 2021 at 2:33 AM Reuven Lax wrote: > > > On Fri, Oct 15, 2021 at 11:13 PM Cristian Constantinescu

Re: Why is Avro Date field using InstantCoder?

2021-10-16 Thread Cristian Constantinescu
e > with avro 1.8? If so, this might be tricky to fix, since Beam maintains > backwards compatibility on its public API. > > On Fri, Oct 15, 2021 at 5:38 PM Cristian Constantinescu > wrote: > >> Hi all, >> >> I've created a small demo project to show the iss

Re: Why is Avro Date field using InstantCoder?

2021-10-15 Thread Cristian Constantinescu
a mapping. I *think* the intention > is that we generate logic for converting Date to/from Instant when making a > getters for a RowWithGetters backed by Avro. > > Brian > > On Thu, Oct 14, 2021 at 4:43 AM Cristian Constantinescu > wrote: > >> A little bit more co

Re: Schemas with uppercase fields in beans

2021-09-28 Thread Cristian Constantinescu
> private void setActionId(String actionId){..} > } > > Brian > > On Sun, Sep 12, 2021 at 9:16 AM Cristian Constantinescu > wrote: > >> Hey all, >> >> I might have found a bug in the way we create schemas from Java Beans. >> >> My schema has to hav

Re: Possible bug in GetterBasedSchemaProvider.fromRowFunction

2021-09-15 Thread Cristian Constantinescu
2021 at 3:06 PM Cristian Constantinescu > wrote: > >> I think I tried that, but can't remember for sure (I'm like 80% sure, >> sorry for the uncertainty, I've been trying many things for various >> problems). And it didn't work. However, if I understand this solution >&g

Re: Possible bug in GetterBasedSchemaProvider.fromRowFunction

2021-09-15 Thread Cristian Constantinescu
to join. Is that right? On Wed, Sep 15, 2021 at 5:56 PM Reuven Lax wrote: > Could you actually fill in the generic type for Iterable? e.g. > Iterable lhs; I think without that, the schema won't match. > > On Wed, Sep 15, 2021 at 2:52 PM Cristian Constantinescu > wrote:

Re: Possible bug in GetterBasedSchemaProvider.fromRowFunction

2021-09-15 Thread Cristian Constantinescu
2021 at 7:01 PM Cristian Constantinescu > wrote: > >> Hello everyone, >> >> As I'm continuing to remove my usage of Row and replacing it with Pojos, >> I'm following the documentation for the CoGroup transform [1]. >> >> As per the documentation, I hav

Schemas with uppercase fields in beans

2021-09-12 Thread Cristian Constantinescu
Hey all, I might have found a bug in the way we create schemas from Java Beans. My schema has to have uppercase field names. So I have something like : @DefaultSchema(JavaBeanSchema.class) public class Foo { private String ACTION_ID; private String getACTION_ID(){..} private void

Re: [PROPOSAL] Stable URL for "current" API Documentation

2021-06-17 Thread Cristian Constantinescu
Big +1 here. In the past few days I've replaced the 2.*.0 part of the google found javadoc url with 2.29.0 more times than I could count. I should have made a pipeline with a session window to count those replacements them though :P On Thu, Jun 17, 2021 at 12:18 PM Robert Bradshaw wrote: > This

Tutorial - How to run a Beam pipeline with Flink on Kubernetes Natively

2021-03-22 Thread Cristian Constantinescu
Hi everyone, I spent the week-end putting the pieces together to run Beam with the Flink Runner on Kubernetes. While I did find very good articles and videos about Beam and Flink and Kubernetes separately, I didn't find one that mixes all three of them in the same pot. So, I wrote a small demo

Re: Beam with Confluent Schema Registry and protobuf

2021-01-12 Thread Cristian Constantinescu
5 [2] and it can be blocked by Beam Avro deps update. > > [1] https://issues.apache.org/jira/browse/BEAM-9330 > [2] https://www.confluent.io/blog/introducing-confluent-platform-5-5/ > > On 9 Jan 2021, at 05:16, Cristian Constantinescu wrote: > > Hi everyone, >

Beam with Confluent Schema Registry and protobuf

2021-01-08 Thread Cristian Constantinescu
Hi everyone, Beam currently has a dependency on older versions of the Confluent libs. It makes it difficult to use Protobufs with the Confluent Schema Registry as ConfluentSchemaRegistryDeserializerProvider only supports Avro. I put up together a very simple project to demo how it can be done

Re: [DISCUSS] Move Avro dependency out of core Beam

2020-09-21 Thread Cristian Constantinescu
All the proposed solutions seem reasonable. I'm not sure if one has an edge over the other. I guess it depends on how cautiously the community would like to move. Maybe it's just my impression, but it seems to me that there are a few changes that are held back for the sake of backwards

Re: [DISCUSS] Move Avro dependency out of core Beam

2020-09-11 Thread Cristian Constantinescu
Hi everyone, PR https://github.com/apache/beam/pull/12748 now passes all the checks, and could potentially be merged (not advocating this, just saying). I've rebased on the latest master as of today. I've also left a comment in the PR with the high level changes for ALL the modules. I encourage

BEAM-9330

2020-08-28 Thread Cristian Constantinescu
Hello, I would like to contribute to https://issues.apache.org/jira/browse/BEAM-9330, getting Beam to support PROTOBUF and JSON schemas with Confluent Schema Registry. I'm a bit of a Java newb (C# and Go are more up my alley) and this would be my first contribution to any ASF projects. As such,