Re: Should UnboundedSource provide a split identifier ?

2016-10-07 Thread Amit Sela
Filed https://issues.apache.org/jira/browse/BEAM-727.

As for initializing from workers, this is actually the opposite of what's
discussed in https://issues.apache.org/jira/browse/BEAM-704.
IMO Kafka metadata should be accessed once, before splitting, since this
metadata eventually determines the splitting (and for other reasons
mentioned in BEAM-704).
As for the "Driver/Launcher" program accessing Kafka, I guess it could run
on a worker, right ? I can only speak for Spark saying that when running on
YARN it can run in "cluster-mode" - running the driver program in a YARN
container as well.

On Tue, Oct 4, 2016 at 10:39 PM Raghu Angadi 
wrote:

> On Wed, Sep 14, 2016 at 1:43 PM, Amit Sela  wrote:
>
> > >
> > > For generateInitialSplits, the UnboundedSource API doesn't require
> > > deterministic splitting (although it's recommended), and a
> PipelineRunner
> > > should keep track of the initially generated splits.
> > >
> > If the splitting were to be consistent, in such way that newly added
> > partitions would be assigned with a new "splitId" while existing ones
> would
> > still be assigned with the same (consistent) splitId, it could support
> > newly added partitions, no ?
>
>
> Yes, consistently assigning the partitions will let us do this. I wouldn't
> hash though, it would not distribute partitions evenly when the number of
> partitions is low (say 2 or 10, which is pretty common case). We can assign
> consistently even with a round-robin assignment. Current assignment would
> work, except when we are reading from multiple topics. We can update it to
> handle multiple topics better (assign each partitions for each topic
> independently).
>
> This strategy would still depend on strong guarantee on
> generateInitialSplits() interface where 'desiredNumSplits' stays same
> across updates.
>
> Please file a jira for adding support for handling change in Kafka
> partitions.
>
> In fact, KafkaIO should probably not fetch partitions info form inside
> generateIntialSplits() at all (sometimes Kafka cluster might not be
> accessible from where the job is launched from) and instead do all the
> initialization from the workers, even though it implies multiple fetches of
> Kafka metadata.
>


Re: [PROPOSAL] Introduce review mailing list and provide update on open discussion

2016-10-07 Thread Maximilian Michels
Hi JB!

> 1. We create a new mailing list: rev...@beam.incubator.apache.org.
> 2. We configure github integration to send all pull request comments on 
> review mailing list. It would allow to track and simplify the way to read the 
> comments and to keep up to date.

I already have it organized that way through filters but having a
dedicated mailing list is a much better idea.

> 3. A technical discussion should be send on dev mailing list with the 
> [DISCUSS] keyword in the subject.
> 4. Once a discussion is open, the author should periodically send an update 
> on the discussion (once a week) >containing a summary of the last exchanges 
> happened on the Jira or github (quick and direct summary).

We can try that on a best-effort basis. Enforcing this seems to be
difficult and could also introduce verbosity on the mailing list.

> 5. Once we consider the discussion close (no update in the last two weeks), 
> the author send a [CLOSE] e-mail on the thread.

I think it is hard to decide when a discussion is closed. Two weeks
seems like a too short amount of time.

In general, +1 for an open development process.

-Max

On Fri, Oct 7, 2016 at 4:05 AM, Jungtaek Lim  wrote:
> +1 except [4] for me, too. [4] may be replaced with linking DISCUSSION mail
> thread archive to JIRA.
> Yes it doesn't update news on discussion to JIRA and/or Github, but at
> least someone needed to see can find out manually.
>
> Thanks,
> Jungtaek Lim (HeartSaVioR)
>
> 2016년 10월 7일 (금) 오전 11:00, Satish Duggana 님이 작성:
>
>> +1 for proposal except for [4]. Agree with Raghu on [4] as it may be
>> burdensome to update with summaries and folks may start replying comments
>> on those summaries etc and conclusions are updated on respective design
>> docs. We may want to start without [4].
>>
>> Thanks,
>> Satish.
>>
>> On Fri, Oct 7, 2016 at 12:00 AM, Raghu Angadi 
>> wrote:
>>
>> > +1 for rev...@beam.incubator.apache.org. Open lists are critically
>> > important.
>> >
>> > My comment earlier was mainly about (4). Sorry about the not being clear.
>> >
>> > On Thu, Oct 6, 2016 at 11:00 AM, Lukasz Cwik 
>> > wrote:
>> >
>> > > +1 for supporting different working styles.
>> > >
>> > > On Thu, Oct 6, 2016 at 10:58 AM, Kenneth Knowles
>> > > >
>> > > wrote:
>> > >
>> > > > +1 to rev...@beam.incubator.apache.org if it is turnkey for infra to
>> > set
>> > > > up, aka points 1 and 2.
>> > > >
>> > > > Even though I would not personally read it via email, getting the
>> > > > information in yet another format and infrastructure (and
>> stewardship)
>> > is
>> > > > valuable for search, archival, and supporting diverse work styles.
>> The
>> > > > benefit might not be huge, but I think it will be enough to justify
>> the
>> > > > (hopefully negligible) cost.
>> > > >
>> > > > Kenn
>> > > >
>> > > > On Thu, Oct 6, 2016 at 4:54 AM Jean-Baptiste Onofré > >
>> > > > wrote:
>> > > >
>> > > > Hi team,
>> > > >
>> > > > following the discussion we had about technical discussion that
>> should
>> > > > happen on the mailing list, I would like to propose the following:
>> > > >
>> > > > 1. We create a new mailing list: rev...@beam.incubator.apache.org.
>> > > > 2. We configure github integration to send all pull request comments
>> on
>> > > > review mailing list. It would allow to track and simplify the way to
>> > > > read the comments and to keep up to date.
>> > > > 3. A technical discussion should be send on dev mailing list with the
>> > > > [DISCUSS] keyword in the subject.
>> > > > 4. Once a discussion is open, the author should periodically send an
>> > > > update on the discussion (once a week) containing a summary of the
>> last
>> > > > exchanges happened on the Jira or github (quick and direct summary).
>> > > > 5. Once we consider the discussion close (no update in the last two
>> > > > weeks), the author send a [CLOSE] e-mail on the thread.
>> > > >
>> > > > WDYT ?
>> > > >
>> > > > Regards
>> > > > JB
>> > > > --
>> > > > Jean-Baptiste Onofré
>> > > > jbono...@apache.org
>> > > > http://blog.nanthrax.net
>> > > > Talend - http://www.talend.com
>> > > >
>> > >
>> >
>>


Jenkins build became unstable: beam_Release_NightlySnapshot #192

2016-10-07 Thread Apache Jenkins Server
See