eases in the past/has credentials for the npm
> package? Maybe @Robert Bradshaw ? We will need a
> token set up as a secret to automate this.
>
> I'll also note that we don't do any typescript validation today, and it
> would be nice to publish RCs as part of this
>
> On Tue, Apr
+1 (binding) The artifacts all look good to me.
On Wed, Apr 3, 2024 at 1:35 PM XQ Hu via dev wrote:
> +1 (non-binding). Tested this using a simple Dataflow ML pipeline:
> https://github.com/google/dataflow-ml-starter/actions/runs/8541848483.
>
> On Wed, Apr 3, 2024 at 2:35 PM Jeff Kinard
lem of not shuffling excess redundant data, and also provides
>>> the vast majority of the functionality that a lambda would, with
>>> significantly better debugability and usability too, since the dynamic
>>> destination pattern string can be in display data,
It looks like we're converging on an optional jinja preprocessing phase to
handle this. I'm in favor of this solution.
On Wed, Mar 20, 2024 at 9:23 AM Robert Bradshaw wrote:
> Thanks. I think this will be a very powerful feature. Left some comments
> on the doc.
>
> On Tue, Mar 19
Given the severity of the breakage, and the simplicity of the workaround,
I'm in favor of a patch release. I think we could do Python-only, which
would make the process even more lightweight.
On Wed, Mar 27, 2024 at 3:48 PM Jeff Kinard wrote:
> Hi all,
>
> Beam 2.55 was released with a bug that
On Wed, Mar 27, 2024 at 10:20 AM Reuven Lax wrote:
> Can the prefix still be generated programmatically at graph creation time?
>
Yes. It's just a property of the transform passed by the user at
configuration time.
> On Wed, Mar 27, 2024 at 9:40 AM Robert Bradshaw
> wrote:
>
ansparent than an arbitrary callable as well
for introspection (for both machine and human that may look at the
resulting pipeline).
> I'm not entirely sure how to address this in a portable context. We might
> simply have to accept the extra overhead when going cross language.
>
> Reuven
>
> O
Thanks for putting this together, it will be a really useful feature to
have.
I am in favor of the string-pattern approaches. I think we need to support
both the {record=..., dest_info=...} and the elide-fields approaches, as
the former is nicer when one has a fixed representation for the
output
n to be functionally equivalent to beam.FlatMap(lambda x: x), but
>>>> that would be a larger change and such behavior might need to be
>>>> consistent across SDKs and documented. Adding a default value is a simpler
>>>> change.
>>>>
>>>&g
IIRC, Java has Flatten.iterables() and Flatten.collections(), the first of
which does what you want.
Giving FlatMap a default arg of lambda x: x is an interesting idea. The
only downside I see is a less clear error if one forgets to provide this
(now mandatory) parameter, but maybe that's low
Thanks. I think this will be a very powerful feature. Left some comments on
the doc.
On Tue, Mar 19, 2024 at 11:53 AM Jeff Kinard wrote:
> Hi all,
>
> I have another quick design doc discussing the syntax for Beam YAML
> templates. This feature would allow a user to create a template pipelines
This looks like an interesting proposal. Thanks! I left some comments
on the doc.
On Tue, Mar 5, 2024 at 8:39 AM Reeba Qureshi wrote:
>
> Hi all!
>
> I am Reeba Qureshi, interested in the "Build out Beam Yaml features" (link)
> for GSoC 2024. I worked with Apache Beam during GSoC 2023 and
On Tue, Feb 27, 2024 at 10:39 AM Jan Lukavský wrote:
>
> On 2/27/24 19:22, Robert Bradshaw via dev wrote:
> > On Mon, Feb 26, 2024 at 11:45 AM Kenneth Knowles wrote:
> >> Pulling out focus points:
> >>
> >> On Fri, Feb 23, 2024 at 7:21 PM Robert Bradshaw
"A DelayTimer sets an instant in ProcessingTime at which point
>>>> computations can continue. Runners will prevent the EventTimer watermark
>>>> from advancing past the set OutputTime until Processing Time has advanced
>>>> to at least the provided instant to execute the tim
On Mon, Feb 26, 2024 at 11:45 AM Kenneth Knowles wrote:
>
> Pulling out focus points:
>
> On Fri, Feb 23, 2024 at 7:21 PM Robert Bradshaw via dev
> wrote:
> > I can't act on something yet [...] but I expect to be able to [...] at some
> > time in the process
On Thu, Feb 22, 2024 at 10:16 AM Robert Bradshaw wrote:
>
> On Thu, Feb 22, 2024 at 9:37 AM Reuven Lax via dev
> wrote:
> >
> > On Thu, Feb 22, 2024 at 9:26 AM Kenneth Knowles wrote:
> >>
> >> Wow I love your input Reuven. Of course "the sour
tly the only "hard" signal to the SDK side that the window's work is
> guaranteed to be over, and remaining state needs to be addressed by the
> transform or be garbage collected. This remains critical for developing a
> good pattern for ProcessingTime timers within a Global Window
Thanks for bringing this up.
My position is that both batch and streaming should wait for
processing time timers, according to local time (with the exception of
tests that can accelerate this via faked clocks).
Both ProcessContinuations delays and ProcessingTimeTimers are IMHO
isomorphic, and
On Thu, Feb 22, 2024 at 9:37 AM Reuven Lax via dev wrote:
>
> On Thu, Feb 22, 2024 at 9:26 AM Kenneth Knowles wrote:
>>
>> Wow I love your input Reuven. Of course "the source" that you are applying
>> backpressure to is often a runner's shuffle so it may be state anyhow, but
>> it is good to
ner. Java serialization is inherently self-update-incompatible.
>
> On Wed, Feb 21, 2024 at 1:35 PM Reuven Lax via dev
> wrote:
>>
>> Is there a fundamental reason we serialize java classes into Flink
>> savepoints.
>>
>> On Wed, Feb 21, 2024
We could consider merging the gradle targets without renaming the
classpaths as an intermediate step.
Optimistically, perhaps there's a small number of classes that we need
to preserve (e.g. SerializablePipelineOptions looks like it was
something specifically intended to be serialized; maybe that
rely
> limiting the state size. However I wouldn't start here - we would want to
> build the simpler implementation first and see how it performs.
>
> On Wed, Feb 21, 2024 at 8:53 AM Robert Bradshaw via dev
> wrote:
>>
>> On Wed, Feb 21, 2024 at 12:48 AM Jan Luka
On Wed, Feb 21, 2024 at 12:48 AM Jan Lukavský wrote:
>
> Hi,
>
> I have left a note regarding the proposed splitting of batch and
> streaming expansion of this transform. In general, a need for such split
> triggers doubts in me. This signals that either
>
> a) the transform does something is
On Wed, Feb 14, 2024 at 10:28 AM Kenneth Knowles wrote:
>
> Hi all,
>
> TL;DR I want to add some API like PTransform.getURN, toProto and fromProto,
> etc. to the Java SDK. I want to do this so that making a PTransform support
> portability is a natural part of writing the transform and not a
week.)
On Fri, Feb 9, 2024 at 6:28 PM Robert Burke wrote:
>
> I can agree to that Robert Bradshaw. Thank you for letting the community know.
>
> (Disclaimer: I am on the Dataflow team myself, but do try to keep my hats
> separated when I'm release manager).
>
> It would be b
I validated that the release artifacts are all correct, tested some simple
Python and Yaml pipelines. Everything is looking good so far.
However, could I ask that you hold this vote open a little longer? We've
got some Dataflow service side changes that relate to 2.54 being the first
release
orts (going back to the days of the
original MapReduce), which is what can make this so much more
efficient.
https://github.com/apache/beam/blob/release-2.21.0/sdks/python/apache_beam/runners/portability/fn_api_runner/translations.py#L669
I am, unfortunately, coming up short in finding good documentation
This is because it allows us to do some of the deduplication before
shuffle via combiner lifting. E.g. say we have [A, A, A, B, B] on one
worker and [B, B, B, B, C, C] on another. Rather than passing all that
data through the GroupByKey (which involves (relatively) expensive
materialization and
> migrate this doc to
> https://github.com/apache/beam/tree/master/contributor-docs
>
> [1] https://s.apache.org/beam-release-vendored-artifacts
>
> On Thu, Jan 18, 2024 at 2:56 PM Robert Bradshaw via dev <
> dev@beam.apache.org> wrote:
>
>> Could you explain th
I think this standard design could still be made to work.
Specifically, the graph would contain a DoFn that has the
RequiresTimeSortedInput bit set, and as a single "subtransform" that
has a different DoFn in its spec that does not require this bit to be
set and whose implementation enforces this
Could you explain the process you used to produce these artifacts?
On Thu, Jan 18, 2024 at 11:23 AM Kenneth Knowles wrote:
> +1
>
> On Wed, Jan 17, 2024 at 6:03 PM Yi Hu via dev wrote:
>
>> Hi everyone,
>>
>>
>> Please review the release of the following artifacts that we vendor:
>>
>> *
Just created https://github.com/apache/beam/pull/29969
On Mon, Jan 8, 2024 at 2:49 PM Robert Bradshaw wrote:
>
> This does appear to be a significant missing feature. I'll try to make
> sure something easier gets in by the next release. See also below.
>
> On Mon, Jan 8, 2024 at
This does appear to be a significant missing feature. I'll try to make
sure something easier gets in by the next release. See also below.
On Mon, Jan 8, 2024 at 11:30 AM Ferran Fernández Garrido
wrote:
>
> Hi Yarden,
>
> Since it's a bounded source you could try with Sql transformation
>
nored in
> non-obvious situations and can (and in my scenario, has) result in
> non-obvious downstream issues.*
I agree this can be confusing. Essentially, Coders are attached to
PCollections (which are assumed to be of homogeneous type) at compile
time.
>
> On Fri, Jan 5, 2024 at 1
On Fri, Jan 5, 2024 at 7:38 AM Joey Tran wrote:
> I've been working with a few data types that are in practice
> unpicklable and I've run into a couple issues stemming from the `Any` type
> hint, which when used, will result in the PickleCoder getting used even if
> there's a coder in the coder
+1 (binding)
Validated the artifacts and some simple Python pipelines in a fresh
install.
On Wed, Jan 3, 2024 at 5:46 PM Robert Burke wrote:
> +1 (binding)
>
> Validated the Go SDK against my own pipleines.
>
> Robert Burke
>
> On Wed, Jan 3, 2024, 7:52 AM Chamikara Jayalath via dev <
>
ran wrote:
> Would it feel too wrong to put it in commo_urns? [1]
>
> [1]
> https://github.com/apache/beam/blob/8de029a412ab3e87ec92caf29818b51dab4ab02d/sdks/python/apache_beam/portability/common_urns.py
>
> On Wed, Dec 20, 2023 at 4:06 PM Robert Bradshaw via dev <
> dev@b
Technically it's not really part of the model so much as an implementation
detail, but it likely does make sense to put somewhere common.
On Wed, Dec 20, 2023 at 12:55 PM Joey Tran
wrote:
> Hey all,
>
> Is there a particular reason we hard code
> "beam:runner:executable_stage:v1" everywhere in
On Wed, Dec 20, 2023 at 8:41 AM Ben San Nicolas via dev
wrote:
> Hi,
>
> I'm looking to make use of https://github.com/apache/beam/issues/23373 so
> I can use a java avro schema with enums xlang from python.
>
> Are there existing ideas on how to implement this?
>
> I tried taking a look and the
that weren't
>>> considered compatible.
>>>
>>> We have a custom resource hint (for specifying whether a transform needs
>>> access to some software license) that we use with our transforms and that's
>>> what was preventing the fusion I was expecting. I'
That is correct. Side inputs give a view of the "whole" PCollection and
hence introduce a fusion-producing barrier. For example, suppose one has a
DoFn that produces two outputs, mainPColl and sidePColl, that are consumed
(as the main and side input respectively) of DoFnB.
Thanks for the nice doc. So it looks like this new code is taken
iff max_batch_duration_secs is set?
On Thu, Dec 7, 2023 at 12:28 PM Jack McCluskey via dev
wrote:
> Hey everyone,
>
> This is long-overdue, but I wanted to share a doc (
>
To use cross language capabilities from a non-release branch you'll
have to build the cross-language bits yourself as well. This can be
done by
(1) Making sure Java (for java dependencies) is installed.
(2) In the top level of the repository, running .//gradlew
Note that Logical types are not as portable (e.g. for cross-langauge use).
On Mon, Dec 4, 2023 at 9:18 AM Alexey Romanenko
wrote:
>
> Did you take a look by chance on
> org.apache.beam.sdk.schemas.Schema.LogicalType? Can it be helpful for your
> case?
>
> On 4 Dec 2023, at 12:02, Mark Zitnik
On Fri, Dec 1, 2023 at 9:13 AM Steven van Rossum via dev
wrote:
>
> Hi all,
>
> I was benchmarking the fastjson2 serialization library a few weeks back for a
> Java pipeline I was working on and was asked by a colleague to benchmark
> binary JSON serialization against Rows for fun. We didn't do
+1 (binding)
The artifacts all look good, as does Python installation into a fresh
environment.
On Thu, Nov 16, 2023 at 2:41 PM Svetak Sundhar via dev
wrote:
> +1 (non binding)
>
> validated on python use cases.
>
>
> Svetak Sundhar
>
> Data Engineer
> s vetaksund...@google.com
>
>
>
> On
+1 to at least setting the log level to higher than info. Some runner
logging (e.g. job started/done) may be useful.
On Tue, Nov 14, 2023 at 9:37 AM Joey Tran wrote:
>
> Hi all,
>
> I just had a workshop to demo beam for people at my company and there was a
> bit of confusion about whether the
Thanks for writing this up! Added some comments to the doc itself.
On Mon, Nov 13, 2023 at 11:01 PM Johanna Öjeling via dev <
dev@beam.apache.org> wrote:
> Thanks - well written! Interesting with the Any type, I learned something
> new. Added a comment.
>
> Johanna
>
> On Mon, Nov 13, 2023 at
Thanks. I added some comments to the doc and open PR.
On Wed, Nov 8, 2023 at 12:44 PM John Casey via dev wrote:
>
> Hi All,
>
> I've written up a design for adding DLQs to existing Beam IOs. It's been
> through a round of reviews with some Dataflow folks at Google, but I'd
> appreciate any
+1 (binding)
Artifacts and signatures look good, validated one of the Python wheels
in a fresh install.
On Fri, Nov 10, 2023 at 7:23 AM Alexey Romanenko
wrote:
>
> +1 (binding)
>
> Java SDK with Spark runner
>
> —
> Alexey
>
> On 9 Nov 2023, at 16:44, Ritesh Ghorse via dev wrote:
>
> +1
ia dev
>> wrote:
>>
>>> There are 2 execution times when using Beam. The first execution is
>>> local, when a pipeline is constructed, and the second is remote on the
>>> runner, processing data.
>>>
>>> Based on what you said, it sounds like y
File write failures should be throwing exceptions that will terminate the
pipeline on failure. (Generally a distributed runner will make multiple
attempts before abandoning the entire pipeline of course.)
Are you seeing files failing to be written but no exceptions being thrown?
If so, this is
On Tue, Oct 31, 2023 at 10:28 AM Jan Lukavský wrote:
>
> On 10/31/23 17:44, Robert Bradshaw via dev wrote:
> > There are really two cases that make sense:
> >
> > (1) We read the event timestamps from the kafka records themselves and
> > have some external knowledg
There are really two cases that make sense:
(1) We read the event timestamps from the kafka records themselves and
have some external knowledge that guarantees (or at least provides a
very good heuristic) about what the timestamps of unread messages
could be in the future to set the watermark.
On Fri, Oct 27, 2023 at 7:50 AM Kellen Dye via dev wrote:
>
> > Auto is hard, because it would involve
> > querying the runner before pipeline construction, and we may not even
> > know what the runner is at this point
>
> At the point where pipeline construction will start, you should have
e mentioned, we need
to respect the value in those SDKs that have expansion service
implementations first.
> On Thu, Oct 26, 2023 at 2:25 AM Robert Bradshaw via dev
> wrote:
>>
>> Dataflow (among other runners) has the ability to "upgrade" running
>> pipelines
Dataflow (among other runners) has the ability to "upgrade" running
pipelines with new code (e.g. capturing bug fixes, dependency updates,
and limited topology changes). Unfortunately some improvements (e.g.
new and improved ways of writing to BigQuery, optimized use of side
inputs, a change in
ally proposing
>>> and what I think Airflow does today).
>>>
>>> I'm pretty much +1 on any sort of automation here, and it certainly can
>>> come in stages :)
>>>
>>> On Mon, Oct 23, 2023 at 1:50 PM Johanna Öjeling via dev <
>>>
validators is the right approach though, documentation/screen share
>>>> of their specific workflow is definitely less useful than having a more
>>>> general outline of how to install the RC and things to look out for when
>>>> testing.
>>>>
>>>&g
On Fri, Oct 13, 2023 at 1:32 PM Joey Tran wrote:
>>
>>
>>
>> On Fri, Oct 13, 2023 at 1:18 PM Robert Bradshaw wrote:
>>>
>>> On Fri, Oct 13, 2023 at 10:08 AM Joey Tran
>>> wrote:
>>>>
>>>> Are there places on the SDK s
expression rather than a column.)
> On Thu, Oct 19, 2023 at 4:00 PM Robert Bradshaw via dev
> wrote:
>>
>> On Thu, Oct 19, 2023 at 12:53 PM Reuven Lax wrote:
>> >
>> > Is the schema Group transform (in Java) something along these lines?
>>
>&g
a smooth upgrade is by far the
> most impactful validation.
>
> This thread honestly makes me want to delete the spreadsheet but maybe
> come up with a guide for downstream projects to validate against an RC.
> Maybe that's an extreme reaction...
>
I would very much be in favor of that
d, Oct 18, 2023 at 1:11 PM Robert Bradshaw via dev
> wrote:
>>
>> Beam Yaml has good support for IOs and mappings, but one key missing
>> feature for even writing a WordCount is the ability to do Aggregations
>> [1]. While the traditional Beam primitive is GroupByKey (an
implicitly summing over all (numeric) fields, and allowing
more verbose, precise specification as well.
> I also used the word
> "Combine" just to connect it to other Beam writings and whatnot.
+1
> On Thu, Oct 19, 2023 at 1:41 PM Robert Bradshaw via dev
> wrote:
> >
On Thu, Oct 19, 2023 at 11:42 AM Jan Lukavský wrote:
>
> On 10/19/23 19:41, Robert Bradshaw via dev wrote:
> > On Thu, Oct 19, 2023 at 10:25 AM Jan Lukavský wrote:
> >> On 10/19/23 18:28, Robert Bradshaw via dev wrote:
> >>> On Thu, Oct 19, 2023 at 9:00 A
On Thu, Oct 19, 2023 at 10:25 AM Jan Lukavský wrote:
>
> On 10/19/23 18:28, Robert Bradshaw via dev wrote:
> > On Thu, Oct 19, 2023 at 9:00 AM Byron Ellis wrote:
> >> Rill is definitely SQL-oriented but I think that's going to be the most
> >> common. Dat
ut are pretty niche)
I'm curious, what are these?
> and I guess pivot tables coming from the spreadsheet world. Does make me
> wonder what Rails' ORM looks like these days (I last used v4), it had some
> aggregation support and was pretty declarative...
>
> On Wed, Oct 18
hough.
Rill seems to be very SQL-based.
> On Wed, Oct 18, 2023 at 1:12 PM Robert Bradshaw via dev
> wrote:
>>
>> Beam Yaml has good support for IOs and mappings, but one key missing
>> feature for even writing a WordCount is the ability to do Aggregations
>>
Beam Yaml has good support for IOs and mappings, but one key missing
feature for even writing a WordCount is the ability to do Aggregations
[1]. While the traditional Beam primitive is GroupByKey (and
CombineValues), we're eschewing KVs in the notion of more schema'd
data (which has some
mlining releases -- so if
>>> there are alot of manual tests that could be automated, would be great
>>> to discover and then look to address.
>>>
>>> On Tue, Oct 17, 2023 at 8:47 AM Robert Bradshaw via dev <
>>> dev@beam.apache.org> wrote:
&g
+1
I would also strongly suggest that people try out the release against their
own codebases. This has the benefit of ensuring the release won't break
your own code when they go out, and stress-tests the new code against
real-world pipelines. (Ideally our own tests are all passing, and this
+1, we should definitely be enabling at least some caching by default
here. Added some comments to the doc.
On Mon, Oct 16, 2023 at 9:27 AM Anand Inguva via dev
wrote:
>
> Hello,
>
> In Python SDK, the user state and side input caching is disabled by default
> for all the runners except FnAPI
d though.
Another option is to make the suffix a uuid rather than a single counter.
(This would still have issues with the first application possibly getting
mixed up with a "different" first application unless it was always
appended.)
> On Fri, Oct 13, 2023 at 12:52 PM Robert Brad
ssages). At
least with the old, intersecting names we can detect this problem
rather than silently give corrupt data.
On Fri, Oct 13, 2023 at 7:15 AM Joey Tran wrote:
> For posterity: https://github.com/apache/beam/pull/28984
>
> On Tue, Oct 10, 2023 at 7:29 PM Robert Bradshaw
> wrote:
&
file extension, for example) which can be useful for some downstream
> use-cases. Rest of the filename will be filled out by the SDK (window, pane
> etc.) to make sure that the files written by different workers do not
> conflict.
>
> Thanks,
> Cham
>
>
>>
>> We actua
You'll probably need to resolve "s3a:///*.parquet" out into a
concrete non-glob filepattern to inspect it this way. Presumably any
individual shard will do. match and open from
https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/io/FileSystems.html
may be useful.
On Wed, Oct 11,
; On Thu, Oct 12, 2023 at 4:01 PM Robert Bradshaw
> wrote:
>
>> Does this change any development practices? E.g. if I clone the repo, I'm
>> assuming I couldn't run "setup.py test" anymore. What about the generated
>> files (like protos, or the yaml definitions cop
Does this change any development practices? E.g. if I clone the repo, I'm
assuming I couldn't run "setup.py test" anymore. What about the generated
files (like protos, or the yaml definitions copied from other parts of the
repo)?
On Thu, Oct 12, 2023 at 12:27 PM Anand Inguva via dev
wrote:
>
f it's a toggle it'd be
>> a very sticky toggle since it'd be easy for PTransforms to accidentally
>> rely on it.
>>
>> On Thu, Oct 5, 2023 at 12:33 PM Robert Bradshaw
>> wrote:
>>
>>> Huh. This used to be a hard error in Java, but I guess it's toggla
On Tue, Oct 10, 2023 at 4:05 PM Chamikara Jayalath
wrote:
>
> On Tue, Oct 10, 2023 at 4:02 PM Robert Bradshaw
> wrote:
>
>> On Tue, Oct 10, 2023 at 3:53 PM Chamikara Jayalath
>> wrote:
>>
>>>
>>> On Tue, Oct 10, 2023 at 3:41 PM Reuven La
On Tue, Oct 10, 2023 at 4:03 PM Robert Bradshaw wrote:
> On Tue, Oct 10, 2023 at 3:41 PM Reuven Lax wrote:
>
>> I suspect some simple pattern templating would solve most use cases.
>>
>
> That's what I'm leaning towards as well.
>
>
>> We probably would want
eral timestamps to consider: the element timestamp, its
window start and/or end, walltime.
> On Tue, Oct 10, 2023 at 3:35 PM Robert Bradshaw
> wrote:
>
>> On Mon, Oct 9, 2023 at 3:09 PM Chamikara Jayalath
>> wrote:
>>
>>> I would say:
>>>
&
well.
>>
>> On Tue, Oct 10, 2023 at 3:35 PM Robert Bradshaw
>> wrote:
>>
>>> On Mon, Oct 9, 2023 at 3:09 PM Chamikara Jayalath
>>> wrote:
>>>
>>>> I would say:
>>>>
>>>> sink:
>>>>
best way to specify a lambda here though.
> Maybe a regex or the name of a Python callable ?
>
I'd rather not require Python for a pure Java pipeline, but some kind of a
pattern template may be sufficient here.
> On Mon, Oct 9, 2023 at 2:06 PM Robert Bradshaw via dev <
> dev@beam.ap
;> For dynamic destinations, I think just making the "path" component
>>> support a lambda that is parameterized by the input should be adequate
>>> since this allows customers to direct files written to different
>>> destination directories.
>>>
>>>
Ks.
>>>
>>> For dynamic destinations, I think just making the "path" component
>>> support a lambda that is parameterized by the input should be adequate
>>> since this allows customers to direct files written to different
>>> destination directories.
>&g
, one could have a full filepattern that
includes format parameters for dynamically computed bits as well as the
shard number, windowing info, etc. (There are pros and cons to this.)
> On Mon, Oct 9, 2023 at 12:37 PM Robert Bradshaw via dev <
> dev@beam.apache.org> wrote:
>
>> Currently t
.On Mon, Oct 9, 2023 at 1:11 PM Robert Burke wrote:
> I'll note that the file "Writes" in the Go SDK are currently an unscalable
> antipattern, because of this exact question.
>
> Aside from carefully examining other SDKs it's not clear how one authors
> a reliable, automatically shardable,
Currently the various file writing configurations take a single parameter,
path, which indicates where the (sharded) output should be placed. In other
words, one can write something like
pipeline:
...
sink:
type: WriteToParquet
config:
path: /beam/filesytem/dest
and
. Playing with the examples, I wasn't positive
> if my runs were actually succeeding or not based on the stdout alone.
>
> [1] https://play.beam.apache.org/?sdk=java=mI7WUeje_r2
> <https://play.beam.apache.org/?sdk=java=mI7WUeje_r2>
> [2] https://play.beam.apache.org/?sdk=python=hIr
+1 (binding)
Verified artifacts and signatures and tested a simple python pipeline in a
fresh environment with a wheel.
On Wed, Oct 4, 2023 at 8:05 AM Ritesh Ghorse via dev
wrote:
> +1 (non-binding) validated Go SDK quickstart and Python Streaming
> quickstart on Dataflow runner.
>
> Thanks!
>
BeamJava and BeamPython have the exact same behavior: transform names
within must be distinct [1]. This is because we do not necessarily know at
pipeline construction time if the pipeline will be streaming or batch, or
if it will be updated in the future, so the decision was made to impose
this
nishBundle.
>>>>>
>>>>> You are right about Flink. In many cases this is fine - if Flink rolls
>>>>> back to the last checkpoint, the watermark will also roll back, and
>>>>> everything stays consistent. So in general, one does not need
ink/flink-docs-master/api/java/org/apache/flink/streaming/api/functions/sink/TwoPhaseCommitSinkFunction.html>
>>> which
>>> waits for a checkpoint. In Beam, this is the reason we introduced
>>> RequiresStableInput. Of course in practice many Flink users don't
Given the interest in the YAML work by multiple parties, we put together
https://s.apache.org/beam-yaml-contribute to more easily coordinate on this
effort. Nothing that surprising--we're going to continue using the standard
lists, github, etc.--but it should help for folks who want to get
On Fri, Sep 22, 2023 at 10:58 AM Jan Lukavský wrote:
> On 9/22/23 18:07, Robert Bradshaw via dev wrote:
>
> On Fri, Sep 22, 2023 at 7:23 AM Byron Ellis via dev
> wrote:
>
>> I've actually wondered about this specifically for streaming... if you're
>> writing a p
ps://github.com/apache/beam-site which
>> appears to be unused which could probably even have different review and
>> committer sets if we wanted?
>>
>> On Thu, Sep 21, 2023 at 3:19 PM Robert Bradshaw via dev <
>> dev@beam.apache.org> wrote:
>>
>>
On Fri, Sep 22, 2023 at 7:23 AM Byron Ellis via dev
wrote:
> I've actually wondered about this specifically for streaming... if you're
> writing a pipeline there it seems like you're often going to want to put
> high fixed cost things like database connections even outside of the bundle
> setup.
Dataflow uses a work-stealing protocol. The FnAPI has a protocol to ask the
worker to stop at a certain element that has already been sent.
On Thu, Sep 21, 2023 at 4:24 PM Joey Tran wrote:
> Writing a runner and the first strategy for determining bundling size was
> to just start with a bundle
TBH, I'm not a huge fan of the wikis either. My ideal flow would be
something like g3doc, and markdown files in github do a reasonable enough
job emulating that. (I don't think the overhead of having to do a PR for
small edits like typos is oneros, as those are super easy reviews to do as
well...)
1 - 100 of 1301 matches
Mail list logo