I’m also interested in the answer to this. This is essential for reading
from many types of data sources.
On Tue, Jun 20, 2023 at 2:57 PM Sam Bourne wrote:
> +dev to see if anyone has any suggestions.
>
> On Fri, Jun 16, 2023 at 5:46 PM Sam Bourne wrote:
>
>> Hello beam community!
>>
>> I’m
>
> - What does the new prototype code look like (hopefully much cleaner)?
>
Instead of a separate pxd file, you just have the existing .py file with
standard typing annotations.
> - How does performance compare to the Cython approach?
>
Good question. I've not been able to find any posts
Hi all,
At work, I recently started playing around with mypyc[1] as a means to
compile our python code to C extensions, and I'm pretty impressed so far.
Pros
- write normal python code with annotations: we're already doing this!
- no need for cython-specific header files that can get out
>
> Thanks Chad I'll take a look at your talk and design to see if there's any
> ideas we can merge.
>
Thanks Brian. My hope is that even if you don't add the complete
scheduling framework, we'll get all the features and hooks we need to build
our toolset without needing to modify beam code
Hi Brian,
We implemented a feature that's similar to this, but with a different
motivation: scheduled tasks. We had the same need of creating batches of
logical elements, but rather than perform SIMD-optimized computations, we
want to produce remotely scheduled tasks. It's my hope that the
> Brian
>
> On Wed, May 19, 2021 at 8:38 PM Chad Dombrova wrote:
>
>> This is a random idea, but the whole file IO system inside Beam would
>> actually be awesome to extract into its own project. IIRC, it’s not
>> particularly tied to Beam.
>>
>&g
This is a random idea, but the whole file IO system inside Beam would
actually be awesome to extract into its own project. IIRC, it’s not
particularly tied to Beam.
I’m not saying this should be done now, but it’s be nice to keep it mind
for a future goal.
-chad
On Wed, May 19, 2021 at 10:23
>
>
Hi,
First of all, this is an area that could use a lot of help, so thank you
Kyle for digging through the trove of tickets to understand all of the user
stories.
> I should have led with this. Someone wanted to mount credentials into
> the SDK harness [1]. So in this particular case
Thanks!
On Tue, Dec 8, 2020 at 6:54 AM Pablo Estrada wrote:
> Hi Chad!
> I've been meaning to review this, I've just not carved up the time. I'll
> try to get back to you this week with some thoughts!
> Thanks!
> -P.
>
> On Wed, Dec 2, 2020 at 10:31 AM Chad Dombrova wr
Hi everyone,
Beam's niche is low latency, high throughput workloads, but Beam has
incredible promise as an orchestrator of long running work that gets sent
to a scheduler. We've created a modified version of Beam that allows the
python SDK worker to outsource tasks to a scheduler, like Kubernetes
>
>
> Another example of an optional annotation is marking a transform to run on
> secure hardware, or to give hints to profiling/dynamic analysis tools.
>
There seems to be a lot of overlap between this idea and Environments. Can
you talk about how you feel they may be different or related?
>> ERROR: py36-lint: could not install deps [-rbuild-requirements.txt]; v
>> =
>> InvocationError('/usr/local/google/home/ajamato/beam/sdks/python/target/.tox/py36-lint/bin/python
>> target/.tox/py36-lint/bin/pip install --retr
> All of these are great suggestions. I think what I really need though is
> some way to figure out how to cleanly install (perhaps reinstalling)
> everything I need to run all these commands. tox, yapf,
>
tox should be the only thing you need to install. After that, tox will
install whatever
I would like to edit it! I have an apache account and I am a committed but
IIRC I could not edit it with my normal credentials.
On Wed, Oct 28, 2020 at 8:02 PM Robert Burke wrote:
> (it's a wiki, so anyone who requests and account can improve it)
>
> On Wed, Oct 28, 2020, 7:4
It’s unfortunate that those instructions don’t include pre-commit, which is
by far the easiest way to do this.
To set it up:
pip install pre-commit
pre-commit install
Install sets up git pre-commit hooks so that it will run yapf and pylint on
changed files every time you commit (you’ll need
. I heard similar user requests for
> passing specific flags to pip before. Supporting a generic way with an
> optional flag would address those requests.
>
>
>>
>> Another alternative is to simply pre-install the dependencies you want
>> (or even just override /etc
Hi all,
We are running into problems trying to use our own pypi mirror with Beam.
For those who are not well versed in the esotera of python package
management, pip provides a few ways to specify urls for the pypi index
server:
- command line
cs/python2-sunset/#dataflow
>>>>>>> >>>>>>>
>>>>>>> >>>>>>> On Fri, May 8, 2020 at 3:52 PM Valentyn Tymofieiev <
>>>>>>> valen...@google.com> wrote:
>>>>>
> Even when running portably, Dataflow still has its own implementation of
> PubSubIO that is switched out for Python's "implementation." (It's actually
> built into the same layer that provides the shuffle/group-by-key
> implementation.) However, if you used the external Java PubSubIO it may not
Hi all,
quick followup question:
> small correction. While the new runner will be available with Beam 2.21,
>> the Cross-Language support will be available in 2.22.
>> There will be limitations in the initial set of connectors you can use
>> with Cross-Lang. But at least you will have something
>
> https://github.com/apache/beam/blob/bfebbd0d16361f61fa40bfdec2f0cb6f943f7c9a/.test-infra/jenkins/CommonJobProperties.groovy#L79-L95
> >
> > Otherwise no concerns from me.
> >
> > On Tue, Oct 22, 2019 at 12:01 PM Chad Dombrova
> wrote:
> >>
> >&g
>
> This probably does not apply yet, does optional mean that opting-in for
> all or none of the precommit hooks? Would contributors have a choice of
> which pre-commits they can opt-in too?
>
Yes, once the hook is installed, individual checks are opt-out and the
design is clearly built around a
>
>
>> +1. Also would not running dmypy would require all contributors to run a
> process all the time? I do not know if this is desired by existing
> contributors, and I am not sure if it will be friendly to new contributors.
>
Pre-commit git hooks are completely opt-in. Developers who choose
on't think speed needs to be a blocker. The productivity gains we'd get
> from reliable type analysis more than offset the cost IMO.
> >> >>
> >> >> On Mon, Mar 2, 2020 at 2:03 PM Luke Cwik wrote:
> >> >>>
> >> >>> Python lint t
>
> I believe that mypy via pre-commit hook will be faster than 10s since it
> only applies to modified files.
>
Correct, with a few caveats:
- pre-commit can be setup to only run if a python file changes. so
modifying a java file won't trigger mypy to run.
- if *any* python file
Good news everyone!
We nearly have the full beam codebase passing in mypy.
As we are now approaching the zero-error event horizon, I'd like to open up
a discussion around enabling mypy in the PythonLint job. Every day or so a
PR is merged that introduces some new mypy errors, so enabling this
Pablo Estrada
>>> wrote:
>>>
>>>> Hi everyone,
>>>>
>>>> Please join me and the rest of the Beam PMC in welcoming a new
>>>> committer: Chad Dombrova
>>>>
>>>> Chad has contributed to the project in multiple ways, i
to rows /
schemas.
Any thoughts on that?
-chad
On Wed, Feb 19, 2020 at 6:00 PM Robert Bradshaw wrote:
> Hopefully this should be resovled by
> https://issues.apache.org/jira/browse/BEAM-9229
>
> On Wed, Feb 19, 2020 at 5:52 PM Chad Dombrova wrote:
> >
> > We are using ex
We are using external transforms to get access to PubSubIO within python.
It works well, but there is one major issue remaining to fix: we have to
build a custom beam with a hack to add the PubSubIO java deps and fix up
the coders. This affects KafkaIO as well. There's an issue here:
+), 21129
> deletions(-)
> >>>>>>> >> > >>>>
> >>>>>>> >> > >>>> For reference
> >>>>>>> >> > >>>>
> >>>>>>> >> > >
>
>
> Not to mention that all the nice work for the type hints will have to be
> redone in the for 3.x.
>
Note that there's a tool for automatically converting type comments to
annotations: https://github.com/ilevkivskyi/com2ann
So don't let that part bother you.
I'm curious what other
In case it's of any use, there's a tool called towncrier[1] to help compile
changelog fragments and compile them at time of delivery.
I came across this when working on the python-attrs[2] project, which has
some good documentation for contributors on how to use it:
Hi all,
Did we come to a consensus on dates and locations for the summits?
Particularly interested in the North America Summit.
Thanks,
-chad
On Tue, Nov 26, 2019 at 7:26 AM Alexey Romanenko
wrote:
> Probably, it would make sense to wait a bit for October (or September)
> dates since the
>
>
> It'd be good if there was a way to only apply to violating (or at
> least changed) lines.
I assumed the first thing we’d do is convert all of the code in one go,
since it’s a very safe operation. Did you have something else in mind?
-chad
>
> On Tue, Jan 21, 20
+1 to autoformatting
Let me add some nuance to that.
The way I see it there are 2 varieties of formatters: those which take the
original formatting into consideration (autopep8) and those which disregard
it (yapf, black).
I much prefer yapf to black, because you have plenty of options to tweak
>
> > I agree with focusing one mypy for now, but I would propose soon after,
> or in parallel if it will be different folks, to work on pytype and enable
> it as a first class citizen similar to mypy. If there will be a large delta
> between the two then we can decide on what to do next.
>
> If
>
> Pytype seems to detect attribute errors that mypy has not, so it acts as a
> kind-of linter in this case.
> Examples:
>
> https://github.com/apache/beam/pull/10528/files#diff-0cb34b4622b0b7d7256d28b1ee1d52fc
>
>
Hi folks,
I agree with Robert that we need to wait and see before making any
decisions, but I do have some opinions about the probable/desired outcome.
I haven't used pytype, but my experience working with mypy over the past
few years -- and following various issues and peps related to it and
On Mon, Dec 16, 2019 at 5:59 PM Pablo Estrada wrote:
> +chad...@gmail.com is this consistent with behavior
> that you observed?
>
I honestly can't recall, sorry. I just remember that while I was testing I
updated sdk version and some logging stopped. I *think* I was missing the
state/message
step forward. As a long fan of strongly typed
>> functional languages, I'm glad to see this change :)
>> >>
>> >> On Wed, Dec 11, 2019 at 9:44 AM Chad Dombrova
>> wrote:
>> >>>
>> >>> Hi all,
>> >>> Robert has di
>
> IIUC, isolated_build=True and the removal of setup.py invocation in the
> current virtualenv should eliminate any Cython output files in the repo,
> and no need for run_tox_cleanup.sh?
>
Correct, that script is deleted in this commit:
s designed to be quite safe.
-chad
On Tue, Nov 5, 2019 at 2:50 PM Chad Dombrova wrote:
> Glad to hear we have such a forward-thinking community!
>
>
> On Tue, Nov 5, 2019 at 2:43 PM Robert Bradshaw
> wrote:
>
>> Sounds like we have consensus. Let's move forward. I'll
Hi Udi,
> Sorry I didn't realize you already had a solution for the shadowing issue
> and BEAM-8572.
>
No worries at all. I haven't had much time to invest into that PR lately
(most of it I did at home on my own time), but I did get past most of the
major issues. You've been working on so many
met Altay wrote:
>
>> Thank you for spending time on this to clarify it for all of us! Much
>> appreciated.
>>
>> On Sun, Nov 10, 2019 at 3:45 PM Chad Dombrova wrote:
>>
>>> Hi all,
>>>
>>>
>>>> The sdist step creates a package that should
After this PR goes in should we revisit breaking up the python tests into
separate jenkins jobs by python version? One of the problems with that
plan originally was that we lost the parallelism that gradle provides
because we were left with only one tox task per jenkins job, and so the
total time
On Thu, Dec 5, 2019 at 12:36 PM Valentyn Tymofieiev
wrote:
Ah nice, so then the workflow would be: download [missing] deps from pypi
> into a long-lived cache directory, then download copy the same deps into
> a short-lived temporary directory, using long-lived cache directory as
> SoT, then
Another way to copy only the deps you care about is to use `pip download`
to do the copy. I believe you can provide the cache dir to `pip download
--find-links` and it will read from that before reading from pypi (you may
also need to set --wheel-dir to the cache dir as well), and thus it acts as
any other testing requirements from ‘*tests_require*’,
> preferably removing the setup_requires option.
> """
>
> I believe we don't rely on the tests_require definition. Removing it might
> break developers running "python setup.py test", but the alterna
e, Nov 26, 2019 at 4:59 PM Chad Dombrova wrote:
>
>> Is setup_requires being used somewhere else, because I'm still getting
>> errors after removing it from sdks/python/setup.py.
>>
>> I removed it from this PR: https://github.com/apache/beam/pull/9959
>>
>>
eggs
> pytest_runner-5.2-py2.7.egg pytest_runner-5.2-py3.5.egg
> pytest_runner-5.2-py3.6.egg pytest_runner-5.2-py3.7.egg README.txt
>
> I'll go ahead and create a PR to remove setup_requires from setup.py.
>
> On Tue, Nov 26, 2019 at 4:16 PM Chad Dombrova wrote:
>
>>
;
>>> This reliably gives me:
>>> OSError: [Errno 17] File exists:
>>> '/usr/local/google/home/ehudm/src/beam/sdks/python/.eggs/pytest_runner-5.2-py2.7.egg'
>>>
>>> If I remove this line from setup.py the error is gone:
>>> setup_requires
t;
>
>
> On Tue, Nov 26, 2019 at 9:14 AM Luke Cwik wrote:
>
>> I also started to see this on PRs that I'm reviewing.
>> BEAM-8793, BEAM-8653, BEAM-8631, BEAM-8249 mention issues with setup.py and
>> egg_info but this looks different then all of those so I filed BEA
Actually, it looks like I'm getting the same error on multiple PRs:
https://scans.gradle.com/s/ihfmrxr7evslw
On Mon, Nov 25, 2019 at 10:26 PM Chad Dombrova wrote:
> Hi all,
> The cython tests started failing on one of my PRs which were succeeding
> before. The error is one that I
Hi all,
The cython tests started failing on one of my PRs which were succeeding
before. The error is one that I've never seen before (separated onto
different lines to make it easier to read):
Caused by: org.gradle.api.GradleException:
Could not copy file
ose and shift key heavy than
>>> _LOGGER like log or _log?
>>>
>>> Also please dedupe with these existing bugs:
>>> https://issues.apache.org/jira/browse/BEAM-3523
>>> https://issues.apache.org/jira/browse/BEAM-1825
>>>
>>> On Thu, Nov 14, 2019
Hi all,
I think it might be good to update the description of the beam docker
images and add some descriptive tags, because searching for "apache beam"
in docker hub does not turn up anything:
https://hub.docker.com/search?q=apache%20beam=image.
I clicked through 10 pages worth and couldn't find
Hi Thomas,
> Will this include the ability for users to configure logging via pipeline
> options?
>
We're working on a proposal to allow pluggable logging handlers that can be
configured via pipeline options. For example, it would allow you to add a
new logging handler for StackDriver or
On Wed, Nov 13, 2019 at 10:52 AM Robert Bradshaw
wrote:
> I would be in favor of using module-level loggers as well.
+1
Hi,
> You can see that each JobMessagesResponse may contain a message *or* a
>> GetJobStateResponse.
>>
>> What’s the intention behind this design?
>>
> I believe this was because a user may want to listen to both job state and
> messages all in one stream.
>
Just to be crystal clear, what's the
Hi all,
> The sdist step creates a package that should be installed into each
> tox environment. If the tox environment has cython when this apache
> beam package is installed, it should be used. Nose (or whatever)
> should then run the tests.
>
I spent some time this weekend trying to
Hi,
Answers inline below,
It's unclear from the nose source[1] whether it's calling build_py
>> and build_ext, or just build_ext. It's also unclear whether the result of
>> that build is actually used. When python setup.py nosetests runs, it runs
>> inside of a virtualenv created by tox, and
?).
> >
> > Ahmet
> >
> > [1]
> https://builds.apache.org/view/A-D/view/Beam/view/All/job/beam_PreCommit_Python_Cron/2008/consoleFull
> > [2]
> https://github.com/apache/beam/blob/master/sdks/python/apache_beam/coders/slow_coders_test.py#L32
> > [3]
> https:
Another potential solution would be to _not_ use the sdist task to build
the tarball and let tox do it. Tox should install cython on supported
platforms before running sdist itself (which it does by default unless you
explicitly provide it with a tarball, which we are doing). This has the
added
by the k8s operator (or other external orchestration tool)
>> or, further down the road, in a completely elastic/dynamic fashion with
>> active execution mode (where Flink would request resources directly from
>> k8s, similar to how it would work on YARN).
>>
>>
>>
> Do you believe that a future mypy plugin could replace pipeline type
> checks in Beam, or are there limits to what it can do?
>
mypy will get us quite far on its own once we completely annotate the beam
code. That said, my PR does not include my efforts to turn PTransforms
into Generics, which
>
> As Beam devs will be gaining more first-hand experience with the tooling,
> we may need to add a style guide/best practices/FAQ to our contributor
> guide to clarify known issues.
>
I'm happy to help out with that, just let me know.
-chad
> +1 for splitting pre-commit tests into smaller modules. However in this
> case we need to run all the small tests periodically and have some combined
> flag or dashboard for regular monitoring. Otherwise we might not run/check
> on big amount of tests.
>
post-commit seems like the best place
Thanks for the follow up, Thomas.
On Mon, Oct 28, 2019 at 7:55 PM Thomas Weise wrote:
> Follow-up for users looking to run portable pipelines on Flink:
>
> After prototyping the generate-jar-file approach for internal deployment
> and some related discussion, the conclusion was that it is too
> Wow, that is an incredible amount of work!
>
Some people meditate. I annotate ;)
I'm definitely of the opinion that there's no viable counterargument to the
> value of types, especially for large or complex codebases.
>
Agreed. That's part of why I waited until I got the whole thing passing
Can we get more aggressive about separating tests into groups by those that
are dependent on other languages and those that are not? I think we could
dramatically reduce our backlog if we didn’t run all of the Java tests
every time a commit is made that only affects python code, and vice versa.
Hi all,
I've been working on a PR to add static typing to the beam python sdk for
the past 4 months or so. This has been an epic journey which has required
chasing down numerous fixes across several other projects (mypy, pylint,
python-future), but the mypy tests are now passing!
I'm not sure
AM Udi Meiri wrote:
> Also note that changing the job DSL doesn't take effect until the "seed"
> job runs. (use the "run seed job" phrase)
>
> On Tue, Oct 22, 2019 at 11:06 AM Chad Dombrova wrote:
>
>> Thanks, I'll look into this. I have a PR I'm build
://jenkinsci.github.io/job-dsl-plugin/#method/javaposse.jobdsl.dsl.helpers.wrapper.WrapperContext.colorizeOutput
>>
>> BTW, our Jenkins is maintained by ASF's Infrastructure team:
>> https://cwiki.apache.org/confluence/display/INFRA/Jenkins
>>
>> On Tue, Oct 22, 2019 at 10:23 AM Chad Dombrov
Hi all,
As a user trying to grok failures in jenkins I think it would be a huge
help to have color output support. This is something that works out of the
box for CI tools like gitlab and travis, and it really helps bring that
21st century feel to your logs :)
There's a Jenkins plugin for
thanks again!
On Mon, Oct 21, 2019 at 1:03 PM Robert Bradshaw wrote:
> I just merged https://github.com/apache/beam/pull/9845 which should
> resolve the issue.
>
> On Mon, Oct 21, 2019 at 12:58 PM Chad Dombrova wrote:
> >
> > thanks!
> >
> > On Mon,
thanks!
On Mon, Oct 21, 2019 at 12:47 PM Kyle Weaver wrote:
> This issue is being tracked at
> https://issues.apache.org/jira/browse/BEAM-8416.
>
> On Mon, Oct 21, 2019 at 9:42 PM Chad Dombrova wrote:
>
>> Hi all,
>> Is anyon
Hi all,
Is anyone else getting these errors in
apache_beam.runners.portability.artifact_service_test.ZipFileArtifactServiceTest?
They seem to be taking two forms:
zipfile.BadZipFile: Bad CRC-32 for file
Hi all,
I've been following the Jira issue on Flink "active" k8s support
(autoscaling based on task resource requirements, IIUC) and there has been
a lot of activity there lately. There are two design docs [2][3] from
different teams and it seems like some good collaboration is going on to
Hi Robert,
> Sounds nice. Is there a design doc (or, perhaps, you could just give an
> example of what this would look like in this thread)?
>
I'll follow up shortly with something. The good news is that this first PR
is quite straightforward and (I think) is independent of the semantics of
how
Hi all,
One of our goals for the portability framework is to be able to assign
different environments to different segments of a pipeline. This is not
possible right now because environments are a concept that really only
exist in the portable runner as protobuf messages: they lack a proper API
Hi all,
I made a PR to update to the latest version of pylint here:
https://github.com/apache/beam/pull/9725
I think it's ready to go, but would love to get a few eyes on it.
-chad
On Thu, Oct 10, 2019 at 10:56 AM Maximilian Michels wrote:
> Our mailing list already has decent traffic. The current solution is
> better for its readability. People would simply adapt to more emails by
> creating a filter or ignoring them.
+1. I find the dev generally informative, but as a
There's a lot of value to switching to pytest even without xdist. Could we
prune back the goals of this first PR to just achieving feature parity with
nose, and make a followup PR for xdist?
-chad
On Mon, Oct 7, 2019 at 12:04 PM Udi Meiri wrote:
>
>
> On Fri, Oct 4, 2019 at 10:3
PR is ready for review: https://github.com/apache/beam/pull/9735
On Mon, Sep 30, 2019 at 12:03 PM Maximilian Michels wrote:
> +1
>
> On 29.09.19 13:44, Chad Dombrova wrote:
> > I’m planning on porting the existing Java coder to Python. Any
> > objections t
Hi Max,
Any progress on this? There are a few talks I really want to review!
-chad
On Wed, Sep 18, 2019 at 12:56 PM Maximilian Michels wrote:
> Hi Rahul,
>
> The Beam Summit committee is working on this at the moment. Stay tuned.
>
> Thanks,
> Max
>
> On 18.09.19 11:39, rahul patwari wrote:
> I have a WiP PR to convert Beam to use pytest, but it's been stalled.
>
What would it take to get it back on track?
> Another nice thing about pytest is that you'll be able to tell which suite
> a test belongs to.
>
pytest has a lot of quality of life improvements over nose. The biggest
and
Hi all,
I've posted a new PR that just splits out the python lint job here:
https://github.com/apache/beam/pull/9706
I'll be running the seed job shortly unless anyone objects.
-chad
On Tue, Oct 1, 2019 at 9:04 PM Chad Dombrova wrote:
> I haven’t used nose’s parallel execution plugin, bu
27, 2019 at 6:51 PM Kenneth Knowles wrote:
> Do things go wrong when nose is configured to use parallel execution?
>
> On Fri, Sep 27, 2019 at 5:09 PM Chad Dombrova wrote:
>
>> By the way, the outcome on this was that splitting the python precommit
>> job into one job per
I’m planning on porting the existing Java coder to Python. Any objections
to that?
-chad
On Sun, Sep 29, 2019 at 1:02 PM Robert Burke wrote:
> +1
>
> I'm happy to whip together the Go SDK version once the encoding has been
> concretely decided.
>
> On Fri, Sep 27, 2019, 6:0
> It would still be a standard coder - the distinction I'm proposing is that
> there are certain coders that _must_ be implemented by a new runner/sdk
> (for example windowedvalue, varint, kv, ...) since they are important for
> SDK - runner communication, but now we're starting to standardize
ution/v1/standard_coders.yaml
>>
>>
>> On Fri, Sep 27, 2019 at 5:17 PM Chad Dombrova wrote:
>>
>>> Are there any dissenting votes to making a BooleanCoder a standard
>>> (portable) coder?
>>>
>>> I'm happy to make a PR to implement a B
es.apache.org/jira/browse/BEAM-7996
> >>
> >> Kenn
> >>
> >> On Fri, Sep 27, 2019 at 12:57 PM Reuven Lax wrote:
> >>>
> >>> Java has one, implemented as a byte coder. My guess is that nobody has
> gotten around to implementin
> Do we have good pypi caching? If not this could add a lot of overhead to
>>> our already-backed-up CI queue. (btw I still think your change is good, and
>>> just makes proper caching more important)
>>>
>>> Kenn
>>>
>>> On Tue, Sep 24, 2019 at
Hi all,
It seems a bit unfortunate that there isn’t a portable way to serialize a
boolean value.
I’m working on porting my external PubsubIO PR over to use the improved
schema-based external transform API in python, but because of this
limitation I can’t use boolean values. For example, this
Hi all,
I'm working to make the CI experience with python a bit better, and my
current initiative is splitting up the giant Python PreCommit job into 5
separate jobs into separate jobs for Lint, Py2, Py3.5, Py3.6, and Py3.7.
Around 11am Pacific time tomorrow I'm going to initiate the seed jobs,
A survey would be a good place to start. This came up in the
python2-sunsetting thread as well: we don't know what versions of python
people are using with Beam, which makes it difficult to answer the question
of support.
-chad
On Mon, Sep 23, 2019 at 2:57 PM Ankur Goenka wrote:
> I agree,
gt; conversation there?
>
> https://lists.apache.org/thread.html/eba6caa58ea79a7ecbc8560d1c680a366b44c531d96ce5c699d41535@%3Cdev.beam.apache.org%3E
>
> On Thu, Sep 19, 2019 at 12:55 PM Chad Dombrova wrote:
>
>> Hi all,
>> I saw it mentioned on another thread that Beam will drop python2
Hi all,
I had a read through this thread in the archives. It occurred before I
joined the mailing list, so I hope that this email connects up with the
thread properly for everyone.
I'd like to respond to the following points:
I believe we are referring to two separate things with support:
> -
Hi all,
I saw it mentioned on another thread that Beam will drop python2 support by
the end of the year, and I'd like to voice my concern over this timeline.
As far as I can tell, Beam's support for python3 is brand new, and based on
the master Jira ticket on this topic [1], there are still at
Hi Pulasthi,
Just to mirror what Cham said, it would be a non-starter to try to use a
Beam IO source in another framework: to make them work, you'd have to build
something that executes them with their expected protocol, and that would
look an awful lot like a Beam runner. It makes more sense to
1 - 100 of 115 matches
Mail list logo