On Tue, Apr 1, 2025 at 12:01 PM Jonathan Hope
wrote:
>
> Unfortunately both databases will be online during this so conflicts could
> occur in either direction. I had previously dug up an answer around modifying
> the JdbcIO here:
> https://stackoverflow.com/questions/56398422/exception-handlin
+1 (binding)
Checked the artifacts and signatures, tested a few Python and YAML pipelines.
I noticed the github branch was named release-2.64 rather than 2.64.0
(as was typically the convention). Was this intentional?
On Fri, Mar 28, 2025 at 11:27 AM Valentyn Tymofieiev via dev
wrote:
>
> +1 (b
I'd suggest a separate thread).
Ack, no concerns, just confirming this was an intentional rather than
accidental change (and it makes sense to me).
> On Fri, Mar 28, 2025 at 4:13 PM Robert Bradshaw via dev
> wrote:
>>
>> +1 (binding)
>>
>> Checked the artifacts
Apache Beam YAML makes heavy use of schemas to both provide
high-level, semantically meaningful transforms and to more painlessly
facilitate mixing and matching transforms across language boundaries.
This works well where we are able to infer the schemas, but requires
painful manual declarations wh
I'm in favor of deprecating this and cleaning it up, but it depends on
usage. I suspect it is low (or possibly non-existent, especially as there's
little upside to moving away from the default). I cc'd user@ just in case
anyone wants to chime in there. This may be a good thing to add to our
release
Congratulations, Vitaly! Welcome, and thanks for all the great work
you've done so far!
On Mon, Mar 24, 2025 at 2:36 PM Yi Hu via dev wrote:
>
> Congratulations Vitaly!
>
> On Mon, Mar 24, 2025 at 3:28 PM Robert Burke wrote:
>>
>> Congratulations Vitaly!
>>
>> On Mon, Mar 24, 2025, 12:26 PM Vita
Sorry, yes, that was the intent. Try it now. Happy to make anyone who
wants to contribute an editor as well.
On Mon, Mar 17, 2025 at 3:32 PM Joey Tran wrote:
>
> Got an access denied on the doc. Is intended to be shared publicly?
>
> On Mon, Mar 17, 2025 at 6:17 PM Robert Brad
I've been thinking a bit about productionizing yaml pipelines, and a
large part of that involves being able to write and run tests.
I've put my thoughts up at https://s.apache.org/beam-yaml-testing ;
comments welcome.
- Robert
Welcome back!
Generally permission to merge is derived directly from your apache
credentials and status as a comitter (which you still are; merit
doesn't expire). I do seem to recall that there was an effort to
remove permissions from inactive accounts simply to reduce the attack
surface (e.g. in
I like option (2) as well.
On Mon, Feb 24, 2025 at 11:00 AM Ahmed Abualsaud via dev
wrote:
>
> +1 to option (2)
>
> On Mon, Feb 24, 2025 at 1:32 PM Danny McCormick via dev
> wrote:
>>
>> Thanks for looking into this! I think I like option (2) for the base
>> transform since it allows us to nor
hink we should prefer a simple base language here with
>>>> higher level capabilities available through transforms when possible. It
>>>> will be a little more verbose, but more readable/searchable/learnable, and
>>>> it will preserve the base simplicity for the bul
d it, you need
>>> it).
>>>
>>> As a rule, I think we should prefer a simple base language here with
>>> higher level capabilities available through transforms when possible. It
>>> will be a little more verbose, but more readable/searchable/learnable, and
>&
Currently our YAML API supports basic streaming, including setting
windowing for aggregations, but there's no way to retrieve the
windowing/timestamp metadata (short of stepping out of YAML proper and
using Python, Java, etc. DoFn). It would probably be quite useful to have a
more native way of get
Thanks, Rohit. I agree with these changes disabling as-yet unused features.
They are small enough that validations of RC1 (which looks good to me so
far) should be directly applicable so we can keep verifying there until RC2
is out.
On Tue, Feb 11, 2025 at 9:01 AM Jack McCluskey via dev
wrote:
>
f the data. (composites are permitted to break
>> this rule, but the core compute primitives never do)
>>
>> And I guess it seemed wrong to call something an "Object" when it was
>> really an object per window. TBH the "P" was probably always a misnomer
PObjects are just PCollections with a single element in them. They were
most useful for side inputs (and the BeamJava equivalent is the various
Views) but weren't really worth the extra baggage in the API otherwise, as
there's not much novel one can do with a PObject vs. a singleton
PCollection.
O
On Thu, Jan 30, 2025 at 4:25 PM Reuven Lax via dev
wrote:
> PCollectionView is the equivalent of PObject. Given that the Beam API
> needed to work with the windowing model, we needed something like a PObject
> that could be windowed. This is what PCollectionView provides.
>
+1, I'd forgotten abo
This approach looks good to me, and tames the somewhat ad-hoc approach
we've had so far.
On Wed, Jan 29, 2025 at 9:14 AM Kenneth Knowles wrote:
> Looks good to me.
>
> I've been told in the past that it is more "pythonic" to have one large
> module with codepaths that are always there but don't
ion, and for Flume (which uses the
>> same optimizer) there additional considerations like
>> geographically-dispersed data and jobs that we don't have to deal with
>> for Dataflow.
>>
>> Nothing beats out the importance of fusion though, and I think
>> transla
re
then executed by worker code) which isn't "optimization" per se but an
important part of the Beam model.
> On Mon, Jan 27, 2025 at 5:05 PM Robert Bradshaw via dev
> wrote:
>>
>> On Mon, Jan 27, 2025 at 1:46 PM Robert Burke wrote:
>> >
>> &g
ably still be good to do someday, as it
unlocks fusion opportunities and such.
- Robert
>> If you submit a workflow through the python SDK to Dataflow, does dataflow
>> do another round of optimizations? Or is the translation.py optimization
>> phases never used when using the
On Mon, Jan 27, 2025 at 1:00 PM Joey Tran wrote:
>
> I heard mention that there is a flatten unzipping optimization implemented by
> some runners. I didn't see that in the python optimizations in
> translations.py[1]. Just curious what this optimization is?
It's done to increase the possibility
On Thu, Jan 23, 2025 at 10:39 AM Kenneth Knowles wrote:
>
> On Tue, Jan 21, 2025 at 7:51 PM Robert Bradshaw via dev
> wrote:
>>
>> On Tue, Jan 21, 2025 at 7:26 AM Kenneth Knowles wrote:
>> >
>> > On Tue, Jan 21, 2025 at 2:35 AM Jan Lukavský wrote:
>
Welcome to the community, Enrique!
I have no idea why the subscriptions aren't working, or how to debug
this. Apache infra would probably have people who would be better at
looking into this, as they run the mailing lists.
On Wed, Jan 22, 2025 at 11:45 AM Pablo Estrada wrote:
>
> Hello team!
> I
On Tue, Jan 21, 2025 at 7:26 AM Kenneth Knowles wrote:
>
> On Tue, Jan 21, 2025 at 2:35 AM Jan Lukavský wrote:
>>
>>
>> On 1/20/25 18:18, Kenneth Knowles wrote:
>>
>> This all sounds good. I will add my standard comment that this hint is a
>> property of the data, not the pipeline logic. So it i
beam/sdk/transforms/PTransform.html#addAnnotation-java.lang.String-byte:A-
>
> On 1/14/25 18:23, Robert Bradshaw via dev wrote:
> > Resource hints were originally conceived as ways to customize the
> > runtime environment (ram, accelerators, I could imagine things like
> > custo
Resource hints were originally conceived as ways to customize the
runtime environment (ram, accelerators, I could imagine things like
custom containers, data dependencies, etc. being useful here too).
They also apply recursively through composite transforms, and get
reflected in the attached enviro
Hi all,
Please join me and the rest of the Beam PMC in welcoming Danny
McCormick as the newest member of the PMC.
Danny has been contributing to Beam for several years now, most
notably in the important and growing ML components of Beam. During
this time he has been one of the most prolific deve
+1 to RowCoder being an implementation detail (and it doesn't make
much sense to parameterize its string encodings, logically it just has
string fields).
It does make sense, however, to augment CsvIO to be able to name an
encoding that is used to decode the bytes of the file (producing
"standard"
ould be better, just thinking it might be preferable
to avoid two distinct ways of doing almost the same thing.
>
> Kenn
>
> On Fri, Dec 13, 2024 at 4:38 PM Robert Bradshaw via dev
> wrote:
>>
>> We already have
>> https://github.com/apache/beam/blob/release-2.4
Thanks, this'll be a great addition to Beam. Left a couple of comments
on the doc.
On Fri, Dec 13, 2024 at 12:40 PM Danny McCormick via dev
wrote:
>
> Thanks - this is exciting! I left a couple comments, but I am a big +1 to
> this effort!
>
> On Fri, Dec 13, 2024 at 3:07 PM XQ Hu via dev wrote
We already have
https://github.com/apache/beam/blob/release-2.40.0/sdks/python/apache_beam/runners/worker/sdk_worker_main.py#L141
that allows arbitrary code to be imported and executed on worker
startup. (Perhaps we could generalize to let it also reference a
function to be called rather than just
Seems reasonable to me.
On Thu, Dec 12, 2024 at 12:27 PM Danny McCormick via dev
wrote:
>
> Hey everyone, I've been working on upgrading our Java version of protobuf to
> protobuf 4 (also needed to keep many other dependencies up to date). As part
> of this, I've found that the AWS v1 KinesisIO
On Thu, Dec 5, 2024 at 4:03 PM Joey Tran wrote:
>
> Is there any kind of multiprocess parallelism built into the Python sdk
> worker? In other words, is there a way for my runner to start a worker and
> have it has multiple cores instead of having one worker per core? I thought I
> saw this cap
I created https://github.com/apache/beam/pull/33094 . We can continue
to iterate on the best way to do this, but it'd be good to make this
at least possible. I added some more justification in the PR
description.
On Mon, Oct 28, 2024 at 9:58 AM Robert Bradshaw wrote:
>
> On Tue, Oct 22, 2024 at 1
col, canceling arbitrary
active threads gets dicy...)
> On Wed, Oct 30, 2024, 6:45 PM Robert Bradshaw via dev
> wrote:
>>
>> No, but you can ask it to split ASAP.
>>
>> On Wed, Oct 30, 2024 at 2:32 PM Joey Tran wrote:
>> >
>> > Hey all,
>> >
No, but you can ask it to split ASAP.
On Wed, Oct 30, 2024 at 2:32 PM Joey Tran wrote:
>
> Hey all,
>
> Is there an RPC for cancelling a processbundlerequest? e.g. if a bundle is
> taking a while and you want it to timeout and start a different bundle?
On Tue, Oct 22, 2024 at 12:36 PM Robert Bradshaw wrote:
>
> On Tue, Oct 22, 2024 at 11:46 AM Danny McCormick
> wrote:
> >
> > > (1a) Provide a special operation "Unnest" that takes a single field
> > > and emits it as the top-level element. This can of course result in
> > > unschema'd PCollectio
Have you considered using the ReadAllFromAvro transform:
https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/extensions/avro/io/AvroIO.ReadAll.html
On Sun, Oct 27, 2024 at 10:41 AM Francesco Scipioni
wrote:
>
> Hi everyone,
>
> I am currently working on implementing a custom I/O
Thanks. It will be a great feature to be able to do (basic) ML in a
low/no-code setting.
On Wed, Oct 16, 2024 at 10:29 AM Jeff Kinard wrote:
>
> Hi all,
>
> As the Beam ML and Beam YAML frameworks and communities continue to grow,
> there is a great opportunity to link these two together by intr
tead of ptransforms.
>>> Another drawback to the stacked combinefn is the label will have to be a
>>> mash of possibly very different combiners squashed into one
>>>
>>> On Fri, Sep 27, 2024 at 3:52 PM Robert Bradshaw via dev <
>>> dev@beam.apache.or
n some SQL dialects. For our dynamic
destinations we chose the keyword "only" to indicate that we want to
only write a specified field (as a top level record) rather than the
entire record.
> On Tue, Oct 22, 2024 at 1:50 PM Robert Bradshaw via dev
> wrote:
>>
>>
ml
).
Yet another option would be to add a yaml StripErrorMetadata
transform, as this is the place where it's not convenient to just do a
map.
>> On Sat, Oct 19, 2024 at 1:03 AM Robert Bradshaw via dev
>> wrote:
>>>
>>> I came across an interesting user repo
I came across an interesting user report at
https://github.com/apache/beam/issues/32866 which made me realize that
providing metadata about a bad element in the "bad records" output is
useful, we don't make it easy to extract the output into a PCollection
of the original elements. The output schema
m a runner optimization perspective, but I think
>>>> it's a lot less ergonomic to publish combinefns instead of ptransforms.
>>>> Another drawback to the stacked combinefn is the label will have to be a
>>>> mash of possibly very different combiners squashed in
I'd be worried about encouraging the anti-pattern of GroupByKey() +
CombinePerGroup() which would make the important (often essential) combiner
lifting optimization harder (pattern detection in the runner vs. composite
detection).
You might also be interested in the TupleCombineFns
https://github
We should definitely make sure there's an issue for this--if this example
isn't running it's unlikely the others are as well.
I'm not sure what playground_snippet does or how it's tested.
On Mon, Sep 23, 2024 at 3:01 PM Efrem Braun
wrote:
> My recollection is that there was a unittest put in pl
On Thu, Sep 19, 2024 at 3:43 PM Joey Tran wrote:
> Ah okay then. I commented out the goavro line and the image does finish
> building. It seems that the python still has be built which takes a bit
> (using ` pip install -e .[gcp,test]` inside the sdks/python directory). It
> looks like the source
't need to adapt the transformer plugins in tsconfig.json,
>>> since, the plugin options are the same as for ttypescript. Hope this works!
>>>
>>> I definitely need to setup my own dev environment. So I can build and
>>> link apache-beam to the starter project
om/search?q=repo%3Aapache%2Fbeam+python3.7+lang%3AMarkdown+&type=code
>>>
>>> Granted, the most popular combo there was not present in this search, so
>>> it's probably not terribly significant, compared to the reason Robert is
>>> guessing.
>>>
Echoing many of the comments here, but organizing them under a single
theme, I would say a good focus for Beam 3.0 could be centering around
being more "transform-centric." Specifically:
- Make it easy to mix and match transforms across pipelines and
environments (SDKs). Key to this will be a push
; > yet because there are (seemingly real) CI failures which I haven't had a
> > chance to deal with yet. - https://github.com/apache/beam/pull/31885.
> >
> > Thanks,
> > Danny
> >
> > On Thu, Aug 8, 2024 at 9:08 PM Robert Bradshaw via dev
&
lease, it ran
> and the resulting PR is currently assigned to me. I have not merged it yet
> because there are (seemingly real) CI failures which I haven't had a chance
> to deal with yet. - https://github.com/apache/beam/pull/31885.
>
> Thanks,
> Danny
>
> On Thu, Aug
I noticed when updating a dependency in the base image requirements and running
./gradlew :sdks:python:container:generatePythonRequirementsAll
that a *lot* of other dependencies changed as well. Most were minor
version bumps, but this seems less than ideal. I wonder if we should
run this script p
+1 (binding)
Validated that the artifacts and signatures are good. Tested a couple of
simple pipelines in a fresh install.
On Mon, Aug 5, 2024 at 3:19 PM Valentyn Tymofieiev via dev <
dev@beam.apache.org> wrote:
> +1. Verified that a cherry-pick I made actually made the difference in the
> new r
We should be validating as much of our documentation as possible in
our testing, e.g.
https://github.com/apache/beam/tree/master/sdks/python/apache_beam/examples/snippets
is where the actual code and examples on our website (for Python)
lives to ensure it is regularly tested (on each PR) doesn't bi
Yeah, that makes a lot of sense to me. (I suppose we could promote
both of them to complexes as well.)
On Fri, Jun 28, 2024 at 8:36 AM Joey Tran wrote:
>
> Hey all,
>
> We're running into a fairly common situation where we get a TypeCheckError
> when we supply an int value where a float paramete
On Mon, Jun 24, 2024 at 9:59 AM Kenneth Knowles wrote:
>
> Step 1 and 2 sound great. (I tried a first step with
> https://github.com/apache/beam/pull/29992 but didn't have bandwidth)
>
> This will make it easier for people to get started with beam without having
> to deal with ancient version co
+1 (binding)
The signatures and artifacts all look good. Also tested out some
pipelines with the Python SDK installed into a fresh virtual
environment.
On Mon, Jun 24, 2024 at 2:20 AM Jan Lukavský wrote:
>
> +1 (binding)
>
> Tested Java SDK with Flink Runner 1.18.
>
> Jan
>
> On 6/22/24 06:43,
Thanks for the update. Lots of great stuff here!
On Thu, Jun 13, 2024 at 8:27 AM Andrey Devyatkin via dev
wrote:
>
> Hey Beam community,
>
> We are glad to announce some new changes that our team has been working on
> for a while to implement new solutions and enhance the existing ones. The
> m
storage. Anyone know
of any good google docs -> markdown converters? That might be harder
to do if there are images (and of course comment threads, etc. would
be less likely to be preserved).
Even snapshotting them as PDF or HTML might not be that bad, if the
primary purpose is for archivi
While I'm not opposed to having a more formal process for proposals to
go from idea to consensus to implementation, I'm not sure how much
this would solve the primary issues we face (discoverability and
durability). But maybe that could be built into the process? At the
very least we could have an
You could let your @Setup method initialize your serviceAPI if (and
only if) it's null. For your tests, pre-initialize it with a Fake, and
otherwise the real thing will get used.
On Sat, May 25, 2024 at 1:27 PM Ritwik Dutta via dev
wrote:
>
> Hi Hu,
>
> Thank you,
> I am testing the serviceAPI c
Correct, I've just been pushing these manually, and lately there haven't
been many changes to push. I'm all for getting these set up as part of the
standard release process.
On Tue, Apr 16, 2024 at 1:22 PM Danny McCormick
wrote:
> I've never published npm artifacts before, but I imagine the hard
+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 wrote:
d solved the
>>> essential problem 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
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, 2024 at 11:53
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
t;> 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
>>>
>>> On Wed, Mar 27, 202
ransparent 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
&
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 r
collection 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.
>>>>
>
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 enou
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 built
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
fferently.
>>>>
>>>> What to call this new timer then? DelayTimer?
>>>>
>>>> "A DelayTimer sets an instant in ProcessingTime at which point
>>>> computations can continue. Runners will prevent the EventTimer watermark
>>>> from advancing pas
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 processing
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 source" that you are applying
> >> backpressure to is of
ich is
> currently 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 time
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 can
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 gi
the
> FlinkRunner. 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 2
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
when the threshold is exceeded, but severely
> 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 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 sh
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 tot
idn't run into any issues.
>
> Robert Burke
> Beam 2.54.0 Release Manager
>
> On 2024/02/10 01:41:12 Robert Bradshaw via dev wrote:
> > I validated that the release artifacts are all correct, tested some simple
> > Python and Yaml pipelines. Everything is looking go
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 where
that pretty much every runner supports (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 s
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 cro
> 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 o
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:
>>
>> * beam-
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 11:30 AM Ferran F
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
> grouping
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 r
+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 <
> dev@bea
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@beam.
1 - 100 of 235 matches
Mail list logo