[Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-20 Thread Pei He
Hi everyone, Here is a proposal to address the following issue: JIRA issue https://issues.apache.org/jira/browse/BEAM-443 Currently, users doesn’t have a consistent way to wait for the pipeline to finish. Different runners have different implementations. For example: 1. DirectRunner have a isBlock

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-21 Thread Pei He
t; 2. blocking - waitForInterruption() - some signal that terminates the > job. > 3. non-blocking. > > My 2¢, > Amit > > On Thu, Jul 21, 2016 at 1:39 AM Pei He wrote: > >> Hi everyone, >> Here is a proposal to address the following issue: >&

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-21 Thread Pei He
; > AssertionErrors) >> > > >> > > On Thu, Jul 21, 2016 at 11:40 AM, Kenneth Knowles >> > > > >> > > wrote: >> > > >> > >> I like this proposal. It makes pipeline.run() seem like a pretty >> normal >> > >

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-07-25 Thread Pei He
(Duration) which may return early if a terminal state > is entered seems useful. I don't know that we need a return value > here, given that we an then query the PipelineResult however we want > once this returns. waitUntilFinished is simply > waitFor(InfiniteDuration). > >>

[Proposal] Pipelines and their executions naming changes.

2016-08-05 Thread Pei He
Hi all, I have a proposal about how we name pipelines and their executions. The purpose is to clarify the differences between the two, have consensus between runners, and unify the implementation. Current states: * PipelineOptions.appName defaults to mainClass name * DataflowPipelineOptions.jobN

Re: Jenkins build is still unstable: beam_PostCommit_MavenVerify » Apache Beam :: Examples :: Java #1489

2016-10-10 Thread Pei He
Looking at the broken tests. On Mon, Oct 10, 2016 at 10:05 AM, Apache Jenkins Server < jenk...@builds.apache.org> wrote: > See MavenVerify/org.apache.beam$beam-examples-java/1489/> > >

Re: [Proposal] Add waitToFinish(), cancel(), waitToRunning() to PipelineResult.

2016-10-12 Thread Pei He
> > +1 for your proposal Pei > > > > > > On Mon, Jul 25, 2016 at 5:54 PM, Pei He > > wrote: > > > > > >> Looks to me that followings are agreed: > > >> (1). adding cancel() and waitUntilFinish() to PipelineResult. > > >> (In

Re: Introduction

2016-10-17 Thread Pei He
There are a list of starter jira issues (labeled starter), that could help new contributors to get familiar with the code base. Feel free to grab and assign to yourself, and start the contribution. https://issues.apache.org/jira/browse/BEAM-407?filter=12315361&jql=project%20%3D%20BEAM%20AND%20lab

[PROPOSAL] "IOChannelFactory" Redesign and Make it Configurable

2016-11-16 Thread Pei He
Hi, I am working on BEAM-59 "IOChannelFactory redesign". The goals are: 1. Support file-based IOs (TextIO, AvorIO) with user-defined file system. 2. Support configuring any user-defined file system. And, I drafted the design proposal in two parts

Re: [PROPOSAL] "IOChannelFactory" Redesign and Make it Configurable

2016-11-17 Thread Pei He
roducing BeamFileSystem and not > using the Hadoop FileSystem interface ? > > Thanks > Regards > JB > > On 11/17/2016 01:09 AM, Pei He wrote: > >> Hi, >> >> I am working on BEAM-59 >> <https://issues.apache.org/jira/browse/BEAM-59> "IOChannelF

Re: [DISCUSS] Graduation to a top-level project

2016-11-22 Thread Pei He
+1, very exciting and looking forward. -- Pei On Tue, Nov 22, 2016 at 11:07 AM, Aljoscha Krettek wrote: > +1 > > I'm quite enthusiastic about the growth of the community and the open > discussions! > > On Tue, 22 Nov 2016 at 19:51 Jason Kuster > wrote: > > > An enthusiastic +1! > > > > In parti

Re: [PROPOSAL] "IOChannelFactory" Redesign and Make it Configurable

2016-11-30 Thread Pei He
stem abstract is fine. > > My point is that we should provide a HadoopFilesystem extension/plugin for > Beam filesystem asap: that would help us to support a good range of > filesystems quickly. > > Just my $0.01 ;) > > Regards > JB > > > On 11/17/2016 08:18 PM, P

Re: [PROPOSAL] "IOChannelFactory" Redesign and Make it Configurable

2016-12-05 Thread Pei He
sing files. Any thoughts / feedback? -- Pei [1] https://docs.google.com/document/d/11TdPyZ9_zmjokhNWM3Id-XJsVG3qel2lhdKTknmZ_7M/edit# On Wed, Nov 30, 2016 at 1:32 PM, Pei He wrote: > Thanks JB for the feedback. > > Yes, we should provide a hadoop.fs.FileSystem adaptor. As you said, it

Re: [PROPOSAL] "IOChannelFactory" Redesign and Make it Configurable

2016-12-06 Thread Pei He
I am happy to hear thoughts and get help from people working on the python sdk. > On Mon, Dec 5, 2016 at 4:41 PM, Pei He wrote: > > > I have received a lot of comments in "Part 1: IOChannelFactory > > Redesign" [1]. And, I have updated the design based on the feed

Re: [PROPOSAL] "IOChannelFactory" Redesign and Make it Configurable

2016-12-13 Thread Pei He
. It all sounds good to me. > > On Tue, Dec 6, 2016 at 12:57 PM, Pei He wrote: > > > Thanks Kenn for the feedback and questions. > > > > I responded inline. > > > > On Mon, Dec 5, 2016 at 7:49 PM, Kenneth Knowles > > wrote: > > > > > I rea

Re: Jenkins build is still unstable: beam_PostCommit_Java_RunnableOnService_Dataflow #1805

2016-12-13 Thread Pei He
Thanks Dan for helping. I looks to me it related to my change to do BatchRequest in GcsUtil.fileSize(). I think BatchRequest is using the default timeout, which is too short. Looking more into this issue. On Tue, Dec 13, 2016 at 3:31 PM, Dan Halperin wrote: > If you look at the console output

Re: Jenkins build is still unstable: beam_PostCommit_Java_RunnableOnService_Dataflow #1805

2016-12-14 Thread Pei He
Jira filed: https://issues.apache.org/jira/browse/BEAM-1153 PR of a forward fix: https://github.com/apache/incubator-beam/pull/1608 PR of a partially rollback fix: https://github.com/apache/incubator-beam/pull/1611 On Tue, Dec 13, 2016 at 4:13 PM, Pei He wrote: > Thanks Dan for helping. &g