On Fri, Mar 22, 2019 at 3:02 PM Ismaël Mejía wrote:
>
> It is good to remind committers of their responsability on the
> 'cleanliness' of the merged code. Github sadly does not have an easy
> interface to do this and this should be done manually in many cases,
> sadly I have seen many committers j
Nice!
Between this and the portability work
(https://github.com/apache/beam/pull/8115), hopefully we'll have a
modern Spark runner soon. Any idea on how hard (or easy?) it will be
to merge those two?
On Fri, Mar 22, 2019 at 9:29 AM Łukasz Gajowy wrote:
>
> Cool. :) Congrats and thank you for yo
First off, huge +1 to a good integration with Arrow and Beam. I think
to fully realize the benefits we need to have deeper integration than
arrow-frame-batches as elements, i.e. SDKs should be augmented to
understand arrow frames as batches of individual elements, each with
(possibly) their own tim
On Fri, Mar 29, 2019 at 12:54 PM Michael Luckey wrote:
>
> Really like the idea of improving here.
>
> Unfortunately, I haven't worked with python on that scale yet, so bear with
> my naive understandings in this regard. If I understand correctly, the
> suggestion will result in a couple of proj
I agree with Ahmet.
Fastavro seems to be well maintained and has good, tested
compatibility. Unless we expect significant performance improvements
in the standard Avro Python package (a significant undertaking, likely
not one we have the bandwidth to take on, and my impression is that
it's histori
A URN defines the encoding.
There are (unfortunately) *two* encodings defined for a Coder (defined
by a URN), the nested and the unnested one. IIRC, in both Java and
Python, the nested one prefixes with a var-int length, and the
unnested one does not.
We should define the spec clearly and have cr
still break update for pipelines forcing users to
>>>>>> update their code to select the non-well known type. If we don't change
>>>>>> the default type->coder mapping, the well known coder will gain little
>>>>>> usage. I think we should fix
he/beam/blob/release-2.12.0/sdks/java/core/src/main/java/org/apache/beam/sdk/coders/LengthPrefixCoder.java#L64
(and I'm sure there's others).
> On Thu, Apr 4, 2019 at 3:25 PM Robert Bradshaw wrote:
>>
>> I don't know why there are two separate copies of
>> stan
On Thu, Apr 4, 2019 at 6:38 PM Lukasz Cwik wrote:
>
> The issue with unbounded tests that rely on triggers/late data/early
> firings/processing time is that these are several sources of non-determinism.
> The sources make non-deterministic decisions around when to produce data,
> checkpoint, an
l Mejía wrote:
>
> > I like the extra delimitation the brackets give, worth the two extra
> > characters to me. More importantly, it's nice to have consistency, and
> > the only way to be consistent with the past is leave them there.
>
> My point with the brackets
if it's technically feasible, I am also in favor of requiring experimental
features to be (per-tag, Python should be updated) opt-in only. We should
probably regularly audit the set of experimental features we ship (I'd say
as part of the release, but that process is laborious enough, perhaps we
sh
meaning of (or disallow some of) TupleCoder[StrUtf8Coder, ...],
LengthPrefixCoder[StrUtf8Coder], and using StringUtf8Coder for IO.
>
> On Fri, Apr 5, 2019 at 12:38 AM Robert Bradshaw wrote:
>>
>> On Fri, Apr 5, 2019 at 12:50 AM Heejong Lee wrote:
>> >
>> > Robert,
On Sat, Apr 6, 2019 at 12:08 AM Kenneth Knowles wrote:
>
>
>
> On Fri, Apr 5, 2019 at 2:24 PM Robert Bradshaw wrote:
>>
>> On Fri, Apr 5, 2019 at 6:24 PM Kenneth Knowles wrote:
>> >
>> > Nested and unnested contexts are two different encodings. Can we j
On Mon, Apr 8, 2019 at 8:04 PM Kenneth Knowles wrote:
>
> On Mon, Apr 8, 2019 at 1:57 AM Robert Bradshaw wrote:
>>
>> On Sat, Apr 6, 2019 at 12:08 AM Kenneth Knowles wrote:
>> >
>> > On Fri, Apr 5, 2019 at 2:24 PM Robert Bradshaw wrote:
>> >&
Congratulations!
On Thu, Apr 11, 2019 at 12:29 PM Michael Luckey wrote:
>
> Congrats and welcome, Boyuan
>
> On Thu, Apr 11, 2019 at 12:27 PM Tim Robertson
> wrote:
>>
>> Many congratulations Boyuan!
>>
>> On Thu, Apr 11, 2019 at 10:50 AM Łukasz Gajowy wrote:
>>>
>>> Congrats Boyuan! :)
>>>
>>
That's a great idea! It would probably be pretty easy to add the
corresponding code snippets to the docs as well.
On Thu, Apr 11, 2019 at 2:00 PM Maximilian Michels wrote:
>
> Hi everyone,
>
> The Python SDK still lacks documentation on state and timers.
>
> As a first step, what do you think abo
I've been out, so coming a bit late to the discussion, but here's my thoughts.
The expansion service absolutely needs to be able to provide the
dependencies for the transform(s) it expands. It seems the default,
foolproof way of doing this is via the environment, which can be a
docker image with a
On Thu, Apr 18, 2019 at 12:23 AM Kenneth Knowles wrote:
>
> For Robert's benefit, I want to point out that my proposal is to support
> femtosecond data, with femtosecond-scale windows, even if watermarks/event
> timestamps/holds are only millisecond precision.
>
> So the workaround once I have t
On Tue, Apr 23, 2019 at 4:20 PM Kenneth Knowles wrote:
>
> On Tue, Apr 23, 2019 at 5:48 AM Robert Bradshaw wrote:
>>
>> On Thu, Apr 18, 2019 at 12:23 AM Kenneth Knowles wrote:
>> >
>> > For Robert's benefit, I want to point out that my proposal i
On Wed, Apr 24, 2019 at 12:21 PM Maximilian Michels wrote:
>
> Good idea to let the client expose an artifact staging service that the
> ExpansionService could use to stage artifacts. This solves two problems:
>
> (1) The Expansion Service not being able to access the Job Server
> artifact staging
Thanks for the meeting summary, Stephan. Sound like you covered a lot of
ground. Some more comments below, adding onto what Max has said.
On Wed, Apr 24, 2019 at 3:20 PM Maximilian Michels wrote:
>
> Hi Stephan,
>
> This is excited! Thanks for sharing. The inter-process communication
> code looks
It seems to me that we can assume that if Beam is running in a Java 11
runtime, any Java 11 features used in the body of a DoFn should just work.
The interesting part will be whether there is anything on the boundary that
changes (e.g. are there changes to type inference rules that make them
strict
;ll add the snippets next week, if
>> > that's fine by y'all.
>>
>> That would be great. The blog posts are a great way to get started with
>> state/timers.
>>
>> Thanks,
>> Max
>>
>> On 11.04.19 20:21, Pablo Estrada wrote:
>> > I&
e DataSet/DataStream APIs too, which is a much
taller task. Let alone wrapping all the libraries one might want to
use that are built on these APIs.
If this is instead integrated at a higher level, you could swap back
and forth between the new Tables API and the existing Python SDK
(including libra
ing a
>> table-like API jobs from/to another kind of API job or embedding third-part
>> libraries into a table-like API jobs via UDF? Could you kindly explain why
>> this would be a problem for Py4J and will not be a problem if expressing the
>> job with DAG?
>>
&
Don't we push java artifacts to maven repositories as part of the RC
process? And completely unvetted snapshots? (Or is this OK because
they are special opt-in apache-only ones?)
I am generally in favor of the idea, but would like to avoid increased
toil on the release manager.
One potential hitc
have that
> > also include a python example, I wrote it in java...
> >
> > Cheers
> >
> > Reza
> >
> > On Thu, 25 Apr 2019 at 04:34, Reuven Lax > <mailto:re...@google.com>> wrote:
> >
> > Well state is still not implemented fo
, however,
unfortunate. We should probably infer that if we can.)
On Thu, Apr 25, 2019 at 4:19 PM Robert Bradshaw wrote:
>
> The desire was to avoid the implicit disallowed combination wart in
> Python (until we could make sense of it), and also ValueState could be
> surprising with resp
https://github.com/apache/beam/pull/8402
On Thu, Apr 25, 2019 at 4:26 PM Robert Bradshaw wrote:
>
> Oh, this is for the indexing example.
>
> I actually think using CombiningState is more cleaner than ValueState.
>
> https://github.com/apache/beam/blob/release-2.12.0/sdks/
m(INDEX_STATE)):
yield element, index.read()
index.add(1)
>
> -Max
>
> On 25.04.19 16:40, Robert Bradshaw wrote:
> > https://github.com/apache/beam/pull/8402
> >
> > On Thu, Apr 25, 2019 at 4:26 PM Robert Bradshaw wrote:
> >>
> >> Oh, thi
s already know ValueState from the Java SDK.
>
Maybe we should deprecate that :)
On 25.04.19 17:12, Robert Bradshaw wrote:
> > On Thu, Apr 25, 2019 at 4:58 PM Maximilian Michels
> wrote:
> >>
> >> I forgot to give an example, just to clarify for others:
> >&
Thanks for all the hard work!
https://dist.apache.org/repos/dist/dev/beam/2.12.0/ seems empty; were
the artifacts already moved?
On Fri, Apr 26, 2019 at 10:31 AM Etienne Chauchot wrote:
>
> Hi,
> Thanks for all your work and patience Andrew !
>
> PS: as a side note, there were 5 binding votes (I
t;>>> CombiningState in examples, and perhaps we should go further and deprecate
>>>> it in Java and not implement it in python. Either way I think we should be
>>>> consistent between Java and Python.
>>>>
>>>> I'm curious what people think ab
IIRC, there was some talk on making 2.12 the next LTS, but the
consensus is to decide on a LTS after having had some experience with
it, not at or before the release itself.
On Fri, Apr 26, 2019 at 3:04 PM Alexey Romanenko
wrote:
>
> Thanks for working on this, Kenn.
>
> Perhaps, I missed this b
On Mon, Apr 29, 2019 at 3:43 AM Reza Rokni wrote:
>
> @Robert Bradshaw Some examples, mostly built out from cases around Timeseries
> data, don't want to derail this thread so at a hi level :
Thanks. Perfectly on-topic for the thread.
> Looping timers, a timer which allows
I'd imagine that most users will continue to debug their pipelines
using a direct runner, and even if the portable runner is used it can
be run in "loopback" mode where the pipeline-submitting process also
acts as the worker(s), so one can output print statements, set
breakpoints, etc. as if it wer
+1 to introducing this Param for consistency (and making the
substitution more obvious), and I think SDF is still new/experimental
enough we can do this. I don't know if we need Spec in addition to
Param and Provider.
On Sat, Apr 27, 2019 at 1:07 AM Chamikara Jayalath wrote:
>
>
>
> On Fri, Apr 2
Thanks for starting this investigation. As mentioned, most of the work
to date has been on feature parity, not performance parity, but we're
at the point that the latter should be tackled as well. Even if there
is a slight overhead (and there's talk about integrating more deeply
with the Flume DAG
apache.org/legal/release-policy.html
>>>>>
>>>>> Indeed, advertising pre-release artifacts is against ASF rules. For
>>>>> example, Flink was asked to remove a link to the Maven snapshot
>>>>> repository from their download page.
>>&g
eleases. Much like the Java snapshots are stored in a separate opt-in
>> >>>> repository.
>> >>>>
>> >>>> Kenn
>> >>>>
>> >>>> On Thu, Apr 25, 2019 at 5:39 AM Maximilian Michels
>> >>>> wrote:
hink that trigger behavior across transforms should
>> actually be part of the model. Or at least be standardized for SDK
>> authors. This would also imply that it is documented for end users.
>>
>> In the end, users do not care about whether it's part of the model or
>
hough I haven't looked at
Reza's BiTemporalStream yet). Much harder to remove things once
they're in. Can we just add a Any and/or LatestCombineFn and use (and
point to) that instead? With the comment that if you're doing
read-modify-write, an add_input may be better.
> [1] https:
Congratulation, and thanks for all the great contributions each one of you
has made to Beam!
On Thu, May 2, 2019 at 5:51 AM Ruoyun Huang wrote:
> Congratulations everyone! Well deserved!
>
> On Wed, May 1, 2019 at 8:38 PM Kenneth Knowles wrote:
>
>> Congrats! All well deserved!
>>
>> Kenn
>>
>
>>>>>>> can't do it with AnyCombineFn. And I think LatestCombineFn is
>>>>>>> dangerous, especially when it comes to CombingState. ValueState is more
>>>>>>> explicit, and I still maintain that it is status quo, modulo
>>
wrote:
>>>>>
>>>>> It was not a portable Flink runner.
>>>>>
>>>>> Thanks all for the thoughts, I will create JIRAs, as suggested, with my
>>>>> findings and send them out
>>>>>
>>>>> On Tue,
freely transmuted.
> 4) SDK sends artifacts to artifact staging service
> 5) Artifact staging service converts any artifacts to types that the
> environment understands
> 6) Environment is started and gets artifacts from the artifact retrieval
> service.
>
> On Wed, Apr 24, 2
, the rc artifacts will be
>> protected by a flag (i.e. --pre) from general consumption.
>>
>> On Tue, Apr 30, 2019 at 3:59 PM Robert Bradshaw wrote:
>>>
>>> On Tue, Apr 30, 2019 at 6:11 PM Ahmet Altay wrote:
>>> >
>>> > This conversati
gt; package some unwanted changes.
We definitely would not want this to be a manual step--I wouldn't
trust myself :).
> On Thu, May 2, 2019 at 5:43 PM Robert Bradshaw wrote:
>>
>> On Thu, May 2, 2019 at 5:24 PM Michael Luckey wrote:
>> >
>> > Thanks Ahmet
h portability, this will be
>> > turned On per default. Quite important finding imho.
>> >
>> > Beam coders should
>> > not be slower than Flink's. They are simple wrapped. It seems Kryo is
>> > simply slower which we could fix by providi
Looking forward to your writeup, Max. In the meantime, some comments below.
From: Lukasz Cwik
Date: Thu, May 2, 2019 at 6:45 PM
To: dev
>
>
> On Thu, May 2, 2019 at 7:20 AM Robert Bradshaw wrote:
>>
>> On Sat, Apr 27, 2019 at 1:14 AM Lukasz Cwik wrote:
>> >
Very excited to see this. In particular, I think this will be very
useful for cross-language pipelines (not just SQL, but also for
describing non-trivial data (e.g. for source and sink reuse).
The proto specification makes sense to me. The only thing that looks
like it's missing (other than possib
ce to find a solution to this.
> I'm cutting pieces just to make inline replies easier to read.
>
> On Tue, Apr 23, 2019 at 9:03 AM Robert Bradshaw wrote:
>>
>> On Tue, Apr 23, 2019 at 4:20 PM Kenneth Knowles wrote:
>> > - WindowFn must receive exactly the data
+1 to making them significantly more compact in most cases.
From: Pablo Estrada
Date: Wed, May 8, 2019 at 11:35 PM
To: dev
> Hello all,
> Some tests in Python have the problem that when they fail, lots of internal
> logging is dumped onto stdout, and we end up having to scroll way up to find
>
From: Reuven Lax
Date: Wed, May 8, 2019 at 10:36 PM
To: dev
> On Wed, May 8, 2019 at 1:23 PM Robert Bradshaw wrote:
>>
>> Very excited to see this. In particular, I think this will be very
>> useful for cross-language pipelines (not just SQL, but also for
>> descr
From: Kenneth Knowles
Date: Thu, May 9, 2019 at 10:05 AM
To: dev
> This is a huge development. Top posting because I can be more compact.
>
> I really think after the initial idea converges this needs a design doc with
> goals and alternatives. It is an extraordinarily consequential model change
ange, I think the practical decision should be for schemas and
> coders to coexist for the time being. However when we start planning Beam
> 3.0, deprecating coders is something I would like to suggest.
>
> On Thu, May 9, 2019 at 7:48 AM Robert Bradshaw wrote:
>>
>> F
From: Kenneth Knowles
Date: Thu, May 9, 2019 at 5:44 PM
To: dev
>> > *Why multiple int types?* The domain of values for these types are
>> > different. For a language with one "int" or "number" type, that's another
>> > domain of values.
>>
>> What is the value in having different domains? If y
ll of that with a RowCoder we understood to designate
the type(s), right?
> On Thu, May 9, 2019 at 10:16 AM Robert Bradshaw wrote:
>>
>> On the flip side, Schemas are equivalent to the space of Coders with
>> the addition of a RowCoder and the ability to materialize to som
From: Chamikara Jayalath
Date: Thu, May 9, 2019 at 7:49 PM
To: dev
> From: Maximilian Michels
> Date: Thu, May 9, 2019 at 9:21 AM
> To:
>
>> Thanks for sharing your ideas for load testing!
>>
>> > According to other contributors knowledge/experience: I noticed that
>> > streaming with KafkaIO
On Thu, May 9, 2019 at 9:32 AM PM Kenneth Knowles wrote:
> From: Robert Bradshaw
> Date: Wed, May 8, 2019 at 3:00 PM
> To: dev
>
>> From: Kenneth Knowles
>> Date: Wed, May 8, 2019 at 6:50 PM
>> To: dev
>>
>> >> The end-of-window, for firing, can
Congratulations, Pablo!
From: Ismaël Mejía
Date: Wed, May 15, 2019 at 10:22 AM
To:
> Congrats Pablo, well deserved, nece to see your work recognized!
>
> On Wed, May 15, 2019 at 9:59 AM Pei HE wrote:
> >
> > Congrats, Pablo!
> >
> > On Tue, May 14, 2019 at 11:41 PM Tanay Tummalapalli
> > wrot
I would strongly suggest new runners adapt the portability runner from
the start, which will be more forward compatible and more flexible
(e.g. supporting other languages). The primary difference is that
rather than wrapping individual DoFns, one wraps a "fused" bundle of
DoFns (called an Executabl
Isn't there an API for concisely computing new fields from old ones?
Perhaps these expressions could contain references to metadata value
such as timestamp. Otherwise,
Rather than withMetadata reifying the value as a nested field, with
the timestamp, window, etc. at the top level, one could let it
Unfortunately the "write" portion of the reshuffle cannot be
parallelized more than the source that it's reading from. In my
experience, generally the read is the bottleneck in this case, but
it's possible (e.g. if the input compresses extremely well) that it is
the write that is slow (which you se
+1 for removing the code given the current state of things.
On Wed, May 15, 2019 at 12:32 AM Ruoyun Huang wrote:
>
> +1
>
> From: Daniel Oliveira
> Date: Tue, May 14, 2019 at 2:19 PM
> To: dev
>
>> Hello everyone,
>>
>> I'm calling for a vote on removing the deprecated Java Reference Runner
>>
(2) seems reasonable.
On Tue, May 14, 2019 at 3:15 AM Udi Meiri wrote:
>
> It seems like pickling of typing types is broken in 3.5 and 3.6, fixed in 3.7:
> https://github.com/python/typing/issues/511
>
> Here are my attempts:
> https://gist.github.com/udim/ec213305ca865390c391001e8778e91d
>
>
> M
On Wed, May 15, 2019 at 1:00 PM Robert Bradshaw wrote:
>>
>> Unfortunately the "write" portion of the reshuffle cannot be
>> parallelized more than the source that it's reading from. In my
>> experience, generally the read is the bottleneck in this case, but
This does bring up an interesting question though. Are runners
violating (the intent of) the spec if they simply abandon/kill workers
rather than gracefully bringing them down (e.g. so that these
callbacks can be invoked)?
On Tue, May 7, 2019 at 3:55 PM Michael Luckey wrote:
>
> Thanks Kenn and R
?
>
>
>
> On Wed, May 15, 2019, 6:51 AM Robert Bradshaw wrote:
>>
>> This does bring up an interesting question though. Are runners
>> violating (the intent of) the spec if they simply abandon/kill workers
>> rather than gracefully bringing them down (e.g. so th
On Wed, May 15, 2019 at 8:51 PM Kenneth Knowles wrote:
>
> On Wed, May 15, 2019 at 3:05 AM Robert Bradshaw wrote:
>>
>> Isn't there an API for concisely computing new fields from old ones?
>> Perhaps these expressions could contain references to metadata value
&g
On Wed, May 15, 2019 at 8:43 PM Allie Chen wrote:
> Thanks all for your reply. I will try each of them and see how it goes.
>
> The experiment I am working now is similar to
> https://stackoverflow.com/questions/48886943/early-results-from-groupbykey-transform,
> which tries to get early results
On Fri, May 17, 2019 at 4:48 PM Jan Lukavský wrote:
>
> Hi Reuven,
>
> > How so? AFAIK stateful DoFns work just fine in batch runners.
>
> Stateful ParDo works in batch as far, as the logic inside the state works for
> absolutely unbounded out-of-orderness of elements. That basically
> (practica
I have no idea about this failure, but it sounds like you've done due
diligence looking into it at this point and it makes sense to ask some
reviewers to take a look at your code which can happen in parallel to
figuring out the root cuase of this kafka issue before it finally gets
submitted.
On Mo
I created https://issues.apache.org/jira/browse/BEAM-7367
On Mon, May 20, 2019 at 3:11 PM Michael Luckey wrote:
>
> This is most likely caused by Merge of
> https://issues.apache.org/jira/browse/BEAM-7349, which was done lately.
>
> Best,
>
> michel
>
> On Mon, May 20, 2019 at 2:49 PM Charith El
t; I see the first two options quite equally good, although the letter one
> is probably more time consuming to implement. But it would bring
> additional feature to streaming case as well.
>
> Thanks for any thoughts.
>
> Jan
>
> On 5/20/19 12:41 PM, Robert Bradshaw wrote:
&g
The primary con I see with this is that the runner must know ahead of
time, when it starts encoding the iterable, whether or not to treat it
as a large one (to also cache the part it's encoding to the data
channel, or at least some kind of pointer to it). With the current
protocol it can be lazy, a
>> StatefulParDo in the first place.
>>>
>>>>
>>>> > Pipelines that fail in the "worst case" batch scenario are likely to
>>>> degrade poorly (possibly catastrophically) when the watermark falls
>>>> behind in streaming mode
ser themselves, should we elevate this to a property of
> (Stateful?)DoFns that the runner can provide? I think a compelling
> argument can be made here that we should.
>
> +1
>
> Jan
>
> On 5/21/19 11:07 AM, Robert Bradshaw wrote:
> > On Mon, May 20, 2019 at 5:24 PM
ng, but I have trouble seeing how an SDK would take
advantage of this (what would trigger the discard of the first page? What
complexity would that incur vs. the marginal benefit (if any)?). Mostly
this feels like a solution in search of a problem. Is there a problem that
you're trying to solv
On Wed, May 22, 2019 at 6:17 PM Maximilian Michels wrote:
> Hi,
>
> Robert and me were discussing on the subject of user-specified
> environments for external transforms [1]. We couldn't decide whether
> users should have direct control over the environment when they use an
> external transform i
On Thu, May 23, 2019 at 11:07 AM Maximilian Michels wrote:
> My motivation was to get rid of the Docker dependency for the Python VR
> tests. Similarly to how we use Python's LOOPBACK environment for
> executing all non-cross-language tests, I wanted to use Java's EMBEDDED
> environment to run th
n which to perform the expansion (though this would
probably not be offered by most, let alone all, expansion services).
> On 23.05.19 11:31, Robert Bradshaw wrote:
> > On Wed, May 22, 2019 at 6:17 PM Maximilian Michels > <mailto:m...@apache.org>> wrote:
> >
> > Hi,
Thanks for writing this up.
I think the justification for adding this to the model needs to be
that it is useful (you have this covered, though some examples would
be nice) and that it's something that can't easily be done by users
themselves (specifically, though it can be (relatively) cheaply do
t outputs on
>> >> checkpoint in sink). That implies that if you don't have sink that is
>> >> able to commit outputs atomically on checkpoint, the pipeline
>> >> execution should be deterministic upon retries, otherwise shadow
>> >> writes
On Fri, May 24, 2019 at 5:32 AM Reuven Lax wrote:
>
> On Thu, May 23, 2019 at 1:53 PM Ahmet Altay wrote:
>>
>>
>>
>> On Thu, May 23, 2019 at 1:38 PM Lukasz Cwik wrote:
>>>
>>>
>>>
>>> On Thu, May 23, 2019 at 11:37 AM Rui Wang wrote:
>
> A few obvious problems with this code:
> 1.
ironment? That would solve the problem of updating the expansion
> > service, although it adds additional complexity for bringing up the
> > environment.
> >
> >
> > Which environment would be used to perform the expansion? I think
> > thi
Sounds like everyone's onboard with the plan. Any chance we could
publish these for the upcoming 2.13 release?
On Wed, Feb 6, 2019 at 6:29 PM Łukasz Gajowy wrote:
>
> +1 to have a registry for images accessible to anyone. For snapshot images, I
> agree that gcr + apache-beam-testing project seem
r.
+1
And, on a pragmatic note, it'd be good to share the port with the
artifact server as well, in which case the job server could say "serve
artifacts to me" without having to worry about any intervening port
forwarding, etc. that sits between it and the sdk.
> On 27.05.19 13:3
I also favor explicit opt-in, especially when you're mixing mature and
new components.
A differently-named, but still published, artifact seems preferable
IMHO to long lived branches. I don't have a handle on how problematic
this would be in practice (e.g. how would a user know to update the
name.
I'm generally in favor of autoformatters, though I haven't looked at
how well this particular one works. We might have to go with
https://github.com/desbma/black-2spaces given
https://github.com/python/black/issues/378 .
On Mon, May 27, 2019 at 10:43 PM Pablo Estrada wrote:
>
> This looks pretty
>>>
>>> On Mon, May 27, 2019 at 3:35 PM Maximilian Michels wrote:
>>> >
>>> > +1
>>> >
>>> > On 27.05.19 14:04, Robert Bradshaw wrote:
>>> > > Sounds like everyone's onboard with the plan. Any chance we could
>>&
ed to
>>> match the logical ordering. Not only might you have several elements with
>>> the same timestamp, but in reality time skew across backend servers can
>>> cause the events to have timestamps in reverse order of the actual
>>> causality order.
>&g
On Fri, May 24, 2019 at 6:57 PM Kenneth Knowles wrote:
>
> On Fri, May 24, 2019 at 9:51 AM Kenneth Knowles wrote:
>>
>> On Fri, May 24, 2019 at 8:14 AM Reuven Lax wrote:
>>>
>>> Some great comments!
>>>
>>> Aljoscha: absolutely this would have to be implemented by runners to be
>>> efficient. W
I'm not quite following what these sizes are needed for--aren't the
benchmarks already tuned to be specific, known sizes? I agree that
this can be expensive; especially for benchmarking purposes a 5x
overhead means you're benchmarking the sizing code, not the pipeline
itself.
Beam computes estimat
are:
> > >> - The product is clearly marked as beta with a big warning.
> > >> - It looks like mostly a single person project. For the same reason
I also strongly prefer not using a fork for a specific setting. Fork will
only have less people looking at it.
> > >>
ys reduces to
> sequence metadata (e.g. offsets).
Offsets within a file, unordered between files seems exactly analogous
with offsets within a partition, unordered between partitions, right?
> On 5/28/19 11:43 AM, Robert Bradshaw wrote:
> > Huge +1 to all Kenn said.
> >
> &
, that ordering of sources is relevant only for
>> (partitioned!) streaming sources and generally always reduces to
>> sequence metadata (e.g. offsets).
>>
>> Jan
>>
>> On 5/28/19 11:43 AM, Robert Bradshaw wrote:
>> > Huge +1 to all Kenn said.
>> >
+1
I validated the artifacts and Python 3.
On Sat, Jun 1, 2019 at 7:45 PM Ankur Goenka wrote:
>
> Thanks Ahmet and Luke for validation.
>
> If no one has objections then I am planning to move ahead without Gearpump
> validation as it seems to be broken from past multiple releases.
>
> Reminder:
One issue with the fully expanded version is that it's so large it's
hard to read.
I think it would be useful to make the ~ entries (at least) clickable
or with hover tool tips. It would be nice to be able to expand columns
individually as well.
On Tue, Jun 4, 2019 at 7:20 AM Melissa Pashniak wr
Until Python 3 support for Beam is officially out of beta and
recommended, I don't think we can tell people to stop using Python 2.
Given that 2020 is just over 6 months away, that seems a short
transition time, so I would guess we'll have to continue supporting
Python 2 sometime into 2020.
A quic
1 - 100 of 1406 matches
Mail list logo