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...)
Very clear now :). Thanks for the fix; it looks good.
On Fri, Sep 15, 2023 at 5:07 PM Joey Tran wrote:
> Ended up just filing a PR [1]
>
> [1] https://github.com/apache/beam/pull/28489
>
>
> On Fri, Sep 15, 2023 at 12:51 PM Joey Tran
> wrote:
>
>> While implementing a runner, we tried
Could you clarify what you mean by annotating the transform?
On Fri, Sep 15, 2023 at 9:57 AM Joey Tran wrote:
> While implementing a runner, we tried annotating a CombineByKey transform.
> I noticed that the annotations for the CBK are then lost in the fusion
> optimization stage when the CBK
I think this is a great library. I'm on the fence of whether it makes sense
to include with Beam proper vs. be a library that builds on top of Beam.
(Would there be benefits of tighter integration? There is the
maintenance/loss of governance issue.) I am definitely not on the side that
the entire
+1 (binding)
Verified the artifacts and signatures, they all look good. Tried a simple
Python pipeline in a fresh install that worked fine. Thanks for putting
this together.
On Thu, Aug 24, 2023 at 4:09 PM Robert Burke wrote:
> Hi everyone,
> Please review and vote on the release candidate #2
On Thu, Aug 24, 2023 at 12:58 PM Chamikara Jayalath
wrote:
>
>
> On Thu, Aug 24, 2023 at 12:27 PM Robert Bradshaw
> wrote:
>
>> I would like to figure out a way to get the stream-y interface to work,
>> as I think it's more natural overall.
>>
>>
+1
I'd love for this information to be accessible programmatically as well
(both directions: extracting parameters from a transform and constructing a
transform from parameters). Making this pattern easy could encourage
compliance.
On Thu, Aug 24, 2023 at 8:54 AM Svetak Sundhar via dev
wrote:
; to make and maybe even less typing for the user. I was originally thinking
> side inputs and metrics would happen outside the loop, but I think you want
> a class and not a closure at that point for sanity.
>
> On Thu, Aug 24, 2023 at 12:02 PM Robert Bradshaw
> wrote:
>
>
it would allow for SQL
>> statements on individual files with projection pushdown supported for
>> things like Parquet which could have some cool and performant data lake
>> applications. I'll probably do a couple of the simpler IOs as
>> well---there's a Swift AWS SDK binding that's pret
Yes, this is a great step forward (both the automation, and the clarified
guidance). Hopefully we can automate virtual everything but the voting
away.
On Thu, Aug 24, 2023 at 8:56 AM Ismaël Mejía wrote:
> Ah excellent, I was not aware it was the case, great to know we are in
> advance !
>
> On
On Tue, Aug 8, 2023 at 9:50 AM Robert Burke wrote:
>
> Either we keep OWNERS and have the review bot use them, or we remove them and
> use the reviews bot config as the single source of truth.
+1. And I don't see any reason we're going to be any better at keeping
them up to date than we have in
+1 (binding)
Verified the signatures and checksums of the artifacts and (somewhat
vacuous) source tarball. Also verified the artifact doesn't leak classes
outside apache/beam/vendor .
It'd be great if we could move this artifact creation and signing to CI,
see
Thanks. Left a few comments on the doc. Looking forward to ARM support.
On Tue, Jul 18, 2023 at 3:59 PM Valentyn Tymofieiev via dev <
dev@beam.apache.org> wrote:
> Hi Celeste,
>
> Thanks for the proposal and researching the options. Using multi-arch
> images seems like a good way to reduce the
+1 (binding)
Verified all the artifacts and signatures, spot-checked docker images, and
tried a simple pipeline against one of the Python wheels in a fresh
install. Thanks for putting this together.
On Thu, Jul 13, 2023 at 2:03 AM Jan Lukavský wrote:
> +1 (binding)
>
> Tested Java SDK with
On Wed, Jun 14, 2023 at 2:05 PM Austin Bennett wrote:
> A few additional thoughts:
>
> * @Anyone --> Should each starter repo allow issues? Or, better to file
> issues in https://github.com/apache/beam/issues ?
>
I'm on the fence. It would make sense to allow issues here, but I'm also
I'm also -1 on introducing another forum, and concur with Alexey that
mailing lists are a (required) deep part of the culture for apache
projects.
If there's something qualitatively and significantly different about
discussions that makes it a better fit, I would consider it. (E.g. IMHO the
> wrote:
>>>
>>>> Is it actually necessary for a PTransform that is configured via the
>>>> Schema mechanism to also be one that uses RowCoder? Those strike me as two
>>>> separate concerns and unnecessarily limiting.
>>>>
>>>> On
We also presented YAML in the lightning talk session at the Beam summit
where it was well received. (No slides, mostly showed examples and talked
about the concept.)
Another useful doc is a list of proposed projects/improvements:
https://s.apache.org/beam-yaml-pipelines-improvements
Also, feel
If you absolutely cannot tolerate concurrency an external locking mechanism
is required. While a distributed system often waits for a work item to fail
before trying it, this is not always the case (e.g. backup workers may be
scheduled and whoever finishes first is determined to be the successful
I went ahead and took care of this today. Should be good to go.
On Fri, Jun 2, 2023 at 10:29 AM Ritesh Ghorse via dev
wrote:
> The preceding steps before this (blog post, releasing on github, website
> updates are done).
> Waiting on this to be completed before sending out the official
>
+1 (binding)
On Tue, May 30, 2023 at 5:42 PM Robert Bradshaw wrote:
> On Tue, May 30, 2023 at 2:01 PM Ritesh Ghorse via dev
> wrote:
>
>> Thanks Danny and Jack! Dataflow containers are up!
>>
>> Only PMC votes count but feel free to test your use cases and vote on
&
On Tue, May 30, 2023 at 2:01 PM Ritesh Ghorse via dev
wrote:
> Thanks Danny and Jack! Dataflow containers are up!
>
> Only PMC votes count but feel free to test your use cases and vote on this
> thread!
>
While we need at least 3 affirmative PMC votes to formally do a release, it
is definitely
you have a Schema we can auto-generate
> the associated builder on the PTransform? Either way, more DRY.
>
> On Tue, May 30, 2023 at 10:59 AM Robert Bradshaw via dev <
> dev@beam.apache.org> wrote:
>
>> +1 to this simplification, it's a historical artifact that provides n
+1 to this simplification, it's a historical artifact that provides no
value.
I would love it if we also looked into ways to auto-generate the
SchemaTransformProvider (e.g. via introspection if a transform takes a
small number of arguments, or uses the standard builder pattern...),
ideally with
On Tue, May 30, 2023 at 10:37 AM Kenneth Knowles wrote:
>
> On Sat, May 27, 2023 at 4:20 PM Stephan Hoyer via dev
> wrote:
>
>> On Fri, May 26, 2023 at 2:59 PM Robert Bradshaw
>> wrote:
>>
>>> Yes, with_hot_key_fanout only performs a single level of fan
Yes, with_hot_key_fanout only performs a single level of fanout. I don't
think fanning out more than this has been explored, but I would imagine
that for most cases the increased IO would negate most if not all of the
benefits.
In particular, note that we already do "combiner lifting" to do as
Done.
On Tue, May 23, 2023 at 7:36 AM Danny McCormick via dev
wrote:
> Hey everyone, as part of automating our release process (see thread here -
> https://lists.apache.org/thread/mw9dbbdjtkqlvs0mmrh452z3jsf68sct), could
> a PMC member please add the infra supplied gpg public key to our release
On Mon, May 15, 2023 at 8:38 AM Moritz Mack wrote:
>
> Hi all,
>
> I was just looking into an old issue again, SerializablePipelineOptions
> calling FileSystems.setDefaultPipelineOptions on deserialization [1]. This
> applies to various runners including Flink and Spark, but not Dataflow as far
Python, and Typescript containers have all been re-built and
>>>>> pushed to Docker Hub. The underlying code did not change, which
>>>>> fortunately
>>>>> means we can dodge having to build an RC4 to fix this issue.
>>>>>
>>>>&g
Thanks for catching this. This does seem severe enough that we need to fix
it before the release.
On Sat, May 6, 2023 at 10:15 PM Chamikara Jayalath via dev <
dev@beam.apache.org> wrote:
> Seems like Python SDK harness containers built for the current RC are
> broken. Please see
On Fri, May 5, 2023 at 6:27 AM Anand Inguva via dev wrote:
>
> >> Is there a significant gain in dropping 3.7 support before the cut?
>
> No, I think it is just a matter of how soon we want to do it.
Absent a compelling reason otherwise, my view would be to just stick
with the statement of
IIRC, we are supporting Python versions until they are out of support.
This would suggest keeping 3.7 in 2.48. (Not that it matters much.) Is
there a significant gain in dropping 3.7 support before the cut?
On Thu, May 4, 2023 at 8:33 AM Jack McCluskey via dev
wrote:
>
> I'd suggest shooting for
If this is acceptable per the release policy, huge +1 to automating
this step (and not limited to java artifacts alone).
On Wed, May 3, 2023 at 1:14 PM Kenneth Knowles wrote:
>
> To Robert: Good point. I didn't click through. There's always the possibility
> that the two branches of the
The artifacts and signatures all look good, and I validated a couple of
Python pipelines in a fresh install.
Assuming all the tests (including the Dataflow ones) pass (modulo the two
mentioned above; seems a fair justification to not block on those) I'm +1
(binding) on this release.
On Wed, Apr
of the groupby should be higher, to be balanced with the
>>> downstream operator.
>>>
>>> On Tue, Apr 18, 2023 at 19:17 Jeff Zhang wrote:
>>>
>>>> Hi Reuven,
>>>>
>>>> It would be better to set parallelism for operators, as
The main concern here seems to be whether using pre-release candidates
would be too disruptive to our workflow. I think this is an easy
hypothesis to test out--we can give using prerelease candidates a try, and
if that indeed turns out to be a problem we can then do the work of
trying to put
+1
On Mon, Apr 17, 2023 at 11:20 AM Chamikara Jayalath via dev <
dev@beam.apache.org> wrote:
> +1
>
> Thanks,
> Cham
>
> On Mon, Apr 17, 2023 at 11:04 AM Kenneth Knowles wrote:
>
>> +1
>>
>> On Fri, Apr 14, 2023 at 1:30 PM Yi Hu via dev
>> wrote:
>>
>>> Please review the release of the
gested, but
> if that's not acceptable then I'll drop one of the people (probably myself
> since I just finished a release and will probably be last up out of the
> group).
>
> Thanks,
> Danny
>
> On Mon, Apr 17, 2023 at 12:34 PM Robert Bradshaw
> wrote:
>
>> I thin
I think it'd be good if the intersection between this list and the PMC had
cardinality greater than 1. Ahmet might be a good person to keep there.
On Mon, Apr 17, 2023 at 9:25 AM Danny McCormick via dev
wrote:
> Yeah, that is part of the proposal. To be clear, our end state would be a
> single
>
> Wilson(Xiaoshuang) Wang
> Sr. Software Engineer
>
>
> On Mon, Mar 13, 2023 at 12:13 PM Robert Bradshaw via dev
> wrote:
>>
>> The FnApiRunner is primarily for tiny jobs (development and testing)
>> and holds all the data in memory.
The FnApiRunner is primarily for tiny jobs (development and testing)
and holds all the data in memory. You'll likely have to run with a
"real" runner to operate over datasets of this size. If you want to
run locally, you can pass --runner=FlinkRunner and (assuming you have
Java installed) it will
Looks like this was taken care of. Thanks, Ritesh!
On Wed, Mar 8, 2023 at 8:48 PM Saifuddin Adenwala wrote:
>
> Greetings !
> Recently I have contributed to the open source apache beam organization and I
> wanted to ask for the review of my pull request .
> Attaching the link of my pull request
es included were from
> https://github.com/apache/beam/pull/25661 and
> https://github.com/apache/beam/pull/25654, both were keeping integration
> tests from running correctly.
Thanks.
> On Fri, Mar 3, 2023 at 2:09 PM Robert Bradshaw wrote:
>>
>> The released ar
at out, and thanks for helping me fix it! We should be
> all set now
>
> On Fri, Mar 3, 2023 at 1:38 PM Robert Bradshaw wrote:
>>
>> It appears your public key is not published in
>> https://dist.apache.org/repos/dist/release/beam/KEYS .
>>
>> On Fri
It appears your public key is not published in
https://dist.apache.org/repos/dist/release/beam/KEYS .
On Fri, Mar 3, 2023 at 8:33 AM Anand Inguva via dev wrote:
>
> +1 (non-binding)
> Tested python wordcount quick start
> https://beam.apache.org/get-started/quickstart-py/ on Direct Runner and
Thanks for pushing this through!
On Wed, Feb 22, 2023 at 10:38 AM Alexey Romanenko
wrote:
>
> Hi all,
>
> As a part of migration the Avro-related classes from Java SDK “core” module
> to a dedicated extension [1] (as it was discussed here [2] and here [3]), two
> important PRs has been merged
o use a
> bot, vs a GH action - I can also spend time to create a custom GH Action that
> accommodates that. But, that might not be worthwhile if the discussed use
> case isn't functionality we even want as part of the project.
>
> On Tue, Feb 21, 2023 at 12:28 PM Robert Bradshaw
On Tue, Feb 21, 2023 at 10:59 AM Kenneth Knowles wrote:
>
> Agree with Robert here. The human connection is important. Can we have a
> behaviorbot that reminds the reviewer to be extra welcoming up front, and
> then thankful afterwards, instead? :-)
+1
> That said, a bot comment would at
+1 (binding)
Validated release artifacts and signatures and tested a couple of
Python pipelines.
On Mon, Feb 13, 2023 at 8:57 AM Alexey Romanenko
wrote:
>
> +1 (binding)
>
> Tested with https://github.com/Talend/beam-samples/
> (Java SDK v8/v11/v17, Spark 3.x runner).
>
> ---
> Alexey
>
> On
Thanks. I added some comments to the doc.
On Mon, Feb 6, 2023 at 1:33 PM Chamikara Jayalath via dev
wrote:
>
> Hi All,
>
> Beam PTransforms are currently primarily identified as operations in a
> pipeline that perform specific tasks. PTransform implementations were
> traditionally linked to
Seams reasonable to me.
On Tue, Feb 7, 2023 at 4:19 PM Luke Cwik via user wrote:
>
> As per [1], the JDK8 and JDK11 containers that Apache Beam uses have stopped
> being built and supported since July 2022. I have filed [2] to track the
> resolution of this issue.
>
> Based upon [1], almost
,
>
> Jiangjie (Becket) Qin
>
>
>
> On Thu, Feb 2, 2023 at 1:23 AM Robert Bradshaw via dev
> wrote:
>>
>> This sounds reasonable to me. One question I have is why a user would
>> prefer to stick with the DataSet API if the DataStream API is
>> a
This sounds reasonable to me. One question I have is why a user would
prefer to stick with the DataSet API if the DataStream API is
available. Would there be any user-visible difference?
On Wed, Feb 1, 2023 at 1:11 AM Becket Qin wrote:
>
> Hi Beam devs,
>
> I'd like to start a discussion about
On Fri, Jan 13, 2023 at 3:54 PM Becket Qin wrote:
>
> Hi Beam devs,
>
> A few of us are currently working on Flink runner to migrate it away from the
> semi-deprecated DataSet API.
Thanks!
> Can someone help grant the following Jira IDs the permission to work on Jira
> issues?
We have
payloadToConfig'
>>>>>>>>
>>>>>>>> What is this schema and what should it look like?
>>>>>>>>
>>>>>>>> Thanks!
>>>>>>>> -Lina
>>>>>>>>
>>>
gt;
> On Fri, Jan 6, 2023 at 11:09 AM Robert Bradshaw via dev
> wrote:
>>
>> Hi all,
>>
>> Welcome to 2023! As last year, the Google Dataflow team is kicking the
>> first week off with a hackathon, and one of the projects proposed this
>> year was to th
Hi all,
Welcome to 2023! As last year, the Google Dataflow team is kicking the
first week off with a hackathon, and one of the projects proposed this
year was to throw together a rust SDK. If you're interested, you can
follow the progress at
https://github.com/kennknowles/beam/tree/rust/sdks/rust
On Thu, Jul 28, 2022 at 5:37 PM Chamikara Jayalath via dev
wrote:
>
> On Thu, Jul 28, 2022 at 4:51 PM Lina Mårtensson wrote:
>>
>> Thanks for the detailed answers!
>>
>> I totally get the points about development & maintenance cost, and,
>> from a user perspective, about getting the performance
On Wed, Dec 28, 2022 at 10:09 AM Byron Ellis wrote:
>
> On Wed, Dec 28, 2022 at 9:49 AM Robert Bradshaw wrote:
>>
>> On Wed, Dec 28, 2022 at 4:56 AM Danny McCormick via dev
>> wrote:
>> >
>> > > Given the increasing importance of multi language p
On Wed, Dec 28, 2022 at 4:56 AM Danny McCormick via dev
wrote:
>
> > Given the increasing importance of multi language pipelines, it does seem
> > that we should expand the capabilities of the DirectRunner or just go all
> > in on FlinkRunner for testing and local / small scale development
>
>
ot; features for
>>>>>> its users like documentation, validation, environments and so on,
>>>>>>
>>>>>> I did a really quick proof-of-viability implementation here:
>>>>>> https://github.com/byronellis/beam/tree/structured-pipe
While Beam provides powerful APIs for authoring sophisticated data
processing pipelines, it often still has too high a barrier for
getting started and authoring simple pipelines. Even setting up the
environment, installing the dependencies, and setting up the project
can be an overwhelming amount
Saving up all the breaking changes until a major release definitely
has its downsides (look at Python 3). The migration path is often as
important (if not more so) than the final destination.
As for this particular change, I would question how the benefit (it's
unclear what the exact benefit
Thanks for looking into this and the careful writeup. I've read the
design doc and it looks great, but have a couple of questions.
(1) Why did you decide on having a single top-level FileWrite
transform whose config is ([common_parameters], [xml-params],
[csv-params], ...) rather than separate
OK, I put this together as a PR:
https://github.com/apache/beam/pull/24037 (I think I've written this
beam proto -> dot code half a dozen times by now...it'll be good to
have it checked in and reusable.)
On Mon, Nov 7, 2022 at 11:06 AM Matt Casters wrote:
>
> Apache Hop pipelines are actually
I've got one I use in Python too (including drilling down into
composites). It's a portable runner. I should clean it up and make it
generally available.
On Mon, Nov 7, 2022 at 9:25 AM Robert Burke wrote:
>
> The Go SDK has a "dot" runner to visualize pipeline protos as a dot graph but
> it's
+1 (binding)
Validated release artifacts and signatures. Tested a Python pipeline
on a clean install.
On Thu, Oct 13, 2022 at 1:22 PM Ritesh Ghorse via dev
wrote:
>
> +1 (non-binding)
> Validated Go SDK Quickstart on Direct and Dataflow runner.
>
> Thanks,
> Ritesh Ghorse
>
> On Thu, Oct 13,
gt;> >
>> > Could this be contributed to Beam?
>
>
> If it would be of broader interest, I would be happy to work on this for the
> Python SDK.
>
> I can share a link to the code with Googlers.
>
> On Mon, Sep 19, 2022 at 10:47 AM Robert Bradshaw wrote:
>>
&
If one of your inputs fits into memory, using side inputs is
definitely the way to go. If neither side fits into memory, the cross
product may be prohibitively large to compute even on a distributed
computing platform (a billion times a billion is big, though I suppose
one may hit memory limits
On Wed, Aug 17, 2022 at 5:15 AM Alexey Romanenko
wrote:
>
> Good point about unanswered SO questions. +1 that we need to improve a
> situation there.
>
> Yes, we may try to stream them to a new dedicated list but it will require
> people here to subscribe to and check regularly one more list
Thanks. I added some comments to the doc.
I agree with Brian that it makes sense to figure out how this
interacts with batched DoFns, as we'll want to migrate to that.
(Perhaps they're already ready to migrate to as a first step?)
On Fri, Aug 12, 2022 at 1:03 PM Brian Hulette via dev
wrote:
>
>
+1 (binding).
I verified the release artifacts and signatures, and ran a couple of
simple Python pipelines.
On Mon, Aug 15, 2022 at 12:40 PM Chamikara Jayalath via dev
wrote:
>
>
>
> On Mon, Aug 15, 2022 at 11:37 AM Kiley Sok wrote:
>>
>> Thanks everyone!
>>
>> @Chamikara Jayalath The Spark
This is a great idea. I would like to approach this from the
perspective of making it easy to provide a catalog of well-defined
transforms for use in expansion services from typical SDKs and also
elsewhere (e.g. for documentation purposes, GUIs, etc.) Ideally
everything about what a transform is
+1 to this proposal. (Interestingly, this was how the very first
prototype of the FnAPI worked, back before we had a working data plane
:).
On Wed, Nov 3, 2021 at 12:10 PM Robert Burke wrote:
>
> Nit: The FnAPI can also send particularly large iterable values over the
> State API instead, (so
+1 (binding). The artifacts and signatures look good to me, as does
the diff from rc1.
On Thu, Oct 7, 2021 at 10:37 AM Brian Hulette wrote:
>
> +1 (non-binding)
>
> I ran a few pipelines with DataflowRunner, including a DataFrame API (Python
> 3.8.6, pandas 1.2.4) pipeline and Java SDK
On Thu, Sep 30, 2021 at 9:25 AM Ke Wu wrote:
>
> Ideally, we do not want to expose anything directly to users and we, as the
> framework and platform provider, separate things out under the hood.
>
> I would expect users to author their DoFn(s) in the same way as they do right
> now, but we
LP the provided encoded bytes, but instead simply window
> wrap them?
>
> Is that right?
>
> On Thu, Sep 9, 2021, 12:58 PM Robert Bradshaw wrote:
>
>> Go doesn't have a (portable, cross-language) runner. It's fine for SDKs
>> to enumerate the coders they understand.
than going into override/custom
> specified soup. (It's not possible to globally override the coders for the
> '[]byte' and 'string' types, nor is there often reason to.)
>
> On Wed, Sep 8, 2021, 2:56 PM Robert Bradshaw wrote:
>
>> On Wed, Sep 8, 2021 at 1:48 PM Jack McCluskey
On Fri, Sep 3, 2021 at 2:40 AM Jan Lukavský wrote:
>
>
> On 9/3/21 1:06 AM, Robert Bradshaw wrote:
> > On Thu, Sep 2, 2021 at 1:03 AM Jan Lukavský wrote:
> >> Hi,
> >>
> >> I had some more time thinking about this and I'll try to recap that.
> &
Evan
>
> On Fri, Jul 30, 2021 at 12:18 Robert Bradshaw wrote:
>>
>> Running with --no_pipeline_type_check also disables any type inference
>> for coders, so in this case (essentially) all your coders will be
>> PickleCoder. You're getting an error here because beam in
Running with --no_pipeline_type_check also disables any type inference
for coders, so in this case (essentially) all your coders will be
PickleCoder. You're getting an error here because beam inferred the
output of MyDoFn to be a str and chose a (more efficient) str coder to
encode its outputs,
Thanks. Filing a JIRA is fine.
I don't think we use either of these dependencies in such a way that
this poses a danger in Beam, but we should probably upgrade them just
to be safe (and avoid any questions, including surfaced by automated
tools). Fortunately the version bumps don't look that big.
Makes sense to me.
On Thu, Jul 29, 2021 at 11:12 AM Andrew Pilloud wrote:
>
> Hi everyone,
>
> Beam's Calcite SqlTransform has been stable for the last few years. We've
> only been adding new methods and I don't expect us to make any breaking
> changes. I believe it is time we drop the
bid a runner to make
>>> use of a simpler implementation? The expansion of Read might be runner
>>> dependent, that is something we do all the time, or am I missing something?
>>>
>>> Jan
>>>
>>> [1] https://issues.apache.org/jira/browse/BEAM-10940
>&
Sharding is typically a physical rather than logical property of the
pipeline, and I'm not convinced it makes sense to add it to Beam in
general. One can already use keys and GBK/Stateful DoFns if some kind
of logical grouping is needed, and adding constraints like this can
prevent opportunities
This makes a lot of sense to me.
On Thu, Jun 17, 2021 at 9:03 AM Brian Hulette wrote:
>
> Hi everyone,
> You may have noticed that our API Documentation could really use some SEO.
> It's possible to search for Beam APIs (e.g. "beam dataframe read_csv" [1] or
> "beam java ParquetIO" [2]) and
+1 to declaring Golang support out of experimental once the Go Modules
issues are solved. I don't think an SDK needs to support every feature
to be accepted, especially now that we can do cross-language
transforms, and Go definitely supports enough to be quite useful. (WRT
streaming, my
ren't necessarily needed).
> Now the next important step would be to make the docker images multi-arch.
> That would be a great contribution if someone is motivated.
>
>
> On Thu, Jan 28, 2021 at 1:47 AM Robert Bradshaw wrote:
>>
>> Cython supports ARM64. The issue
That would be great! I don't know much about this particular issue,
but tips for getting started in general can be found at
https://beam.apache.org/contribute/
On Thu, Jun 3, 2021 at 10:55 AM Raphael Sanamyan
wrote:
>
> Hi, community,
>
> I would like to start work on this task beam-10396, I
+1 (binding)
Verified the signatures are all good and the source tarball matches github.
On Thu, Jun 3, 2021 at 3:38 PM Ahmet Altay wrote:
>
> +1 (binding) - I ran python quickstart examples on the direct runner.
>
> Thank you for preparing the RC!
>
> Ahmet
>
> On Thu, Jun 3, 2021 at 2:58 PM
+1. It was my understanding as well that consensus was that timers
must be delivered in timestamp order, and "within bundle"
resetting/clearing of timers should be respected (as if each timer was
in its own bundle).
On Wed, May 19, 2021 at 3:01 PM Kenneth Knowles wrote:
>
> Reading over the
+1, definitely think display data more often belongs to composites
than leafs. Dataflow is moving to a model where it accepts Beam protos
directly; hopefully we can get that information to the UI.
On Thu, May 13, 2021 at 4:47 PM Valentyn Tymofieiev wrote:
>
> I also happened to look at display
re sorted by "key2". The
>>> downstreaming process, for example, will make a rolling window with size N
>>> that reads N records together at one time. But note, the rolling window
>>> will not cross different "key1".
>>>
>>> So that is
101 - 200 of 1309 matches
Mail list logo