+1

I definitely think it is important to support spark 1 and 2 simultaneously,
and I agree that side-by-side seems the best way to do it. I'll refrain
from commenting on the specific technical aspects of the two runners and
focus just on the split: I am also curious about the answer to Dan's
question about what code is likely to be shared, if any.

On Thu, Aug 4, 2016 at 9:40 AM, Dan Halperin <[email protected]>
wrote:

> Can they share any substantial code? If not, they will really be separate
> runners.
>
> If so, would it make more sense to fork into runners/spark and
> runners/spark2?
>
>
>
> On Thu, Aug 4, 2016 at 9:33 AM, Ismaël Mejía <[email protected]> wrote:
>
> > +1
> >
> > In particular for three reasons:
> >
> > 1. The new DataSet API in spark 2 and the new semantics it allows for the
> > runner (and the effect that we cannot retro port this to the spark 1
> > runner).
> > 2. The current performance regressions in spark 2 (another reason to keep
> > the spark 1 runner).
> > 3. The different dependencies between spark versions (less important but
> > also a source of runtime conflicts).
> >
> > Just two points:
> > 1.  Considering the alpha state of the Structured Streaming API and the
> > performance regressions I consider that it is important to preserve the
> > previous TransformTranslator in the spark 2 runner, at least until spark
> 2
> > releases some stability fixes.
> > 2. Porting Read.Bound to the spark 1 runner is a must, we must guarantee
> > the same IO compatibility in both runners to make this ‘split’ make
> sense.
> >
> > Negative points of the proposal:
> > - More maintenance work + tests to do, but still worth at least for some
> > time given the current state.
> >
> > Extra comments:
> >
> > - This means that we will have two compatibility matrix columns now (at
> > least while we support spark 1) ?
> > - We must probably make clear for users the advantages/disadvantages of
> > both versions of the runner, and make clear that the spark 1 runner will
> be
> > almost on maintenance mode (with not many new features).
> > - We must also decide later on to deprecate the spark 1 runner, this will
> > depend in part of the feedback from users + the progress/adoption of
> spark
> > 2.
> >
> > Ismaël
> >
> > On Thu, Aug 4, 2016 at 8:39 AM, Amit Sela <[email protected]> wrote:
> >
> > > After discussions with JB, and understanding that a lot of companies
> > > running Spark will probably run 1.6.x for a while, we thought it would
> > be a
> > > good idea to have (some) support for both branches.
> > >
> > > The SparkRunnerV1 will mostly support Batch, but could also support
> > > “KeyedState” workflows and Sessions. As for streaming, I suggest to
> > > eliminate the awkward
> > > <https://github.com/apache/incubator-beam/tree/master/
> > > runners/spark#streaming>
> > > way it uses Beam Windows, and only support Processing-Time windows.
> > >
> > > The SparkRunnerV2 will have a batch/streaming support relying on
> > Structured
> > > Streaming and the functionality it provides, and will provide in the
> > > future, to support the Beam model best as it can.
> > >
> > > The runners will exist under “runners/spark/spark1” and
> > > “runners/spark/spark2”.
> > >
> > > If this proposal is accepted, I will change JIRA tickets according to a
> > > proposed roadmap for both runners.
> > >
> > > General roadmap:
> > >
> > >
> > > SparkRunnerV1 should mostly “cleanup” and get rid of the
> Window-mocking,
> > > while specifically declaring Unsupported where it should.
> > >
> > > Additional features:
> > >
> > >    1.
> > >
> > >    Read.Bound support - actually supported in the SparkRunnerV2 branch
> > that
> > >    is at work and it already passed some tests by JB and Ismael from
> > > Talend.
> > >    I’ve also asked Michael Armbrust from Apache Spark to review this,
> and
> > > once
> > >    it’s all set I’ll backport it to V1 as well.
> > >    2.
> > >
> > >    Consider support for “Keyed-State”.
> > >    3.
> > >
> > >    Consider support for “Sessions”
> > >
> > >
> > > SparkRunnerV2 branch <https://github.com/apache/
> incubator-beam/pull/495>
> > > is
> > > at work right now and I hope to have it out supporting (some)
> event-time
> > > windowing, triggers and accumulation modes for streaming.
> > >
> > > Thanks,
> > > Amit
> > >
> >
>

Reply via email to