Hi,

I am in favor of removing the Stream (or Streaming) suffixes and prefixes.
I think that Gyula was also referring to those.

I think the naming of the Tasks, and user facing operators
(SingleOutputStreamOperator and alike) are fine.

As for the other bunch of Operators we could name them Drivers to be mostly
in line with the batch naming. By the way, most of the classes do not have
"Operator" in their name currently - e.g. the one encapsulating the map
functionality is called StreamMap, however the base classes (StreamOperator
and ChainableStreamOperator) have it in their name explicitly. I could go
with MapDriver instead of StreamMap, ChainableStreamOperator will be
eliminated anyway - StreamOperator needs a new name then: worst case
scenario PactDriver. :)

As for n-ary operators I agree with Gyula.

On Sat, May 9, 2015 at 4:44 PM, Aljoscha Krettek <aljos...@apache.org>
wrote:

> Which name changes are you referring to? The proposed names in my
> recent PR? Or the dropping of Stream from all the classes. For the
> rest I was just rambling about how I don't like the names in the batch
> API. :D
>
> On Fri, May 8, 2015 at 12:31 PM, Gyula Fóra <gyula.f...@gmail.com> wrote:
> > Generally I am in favor of making these name changes. My only concern is
> > regarding to the one-input and multiple inputs operators.
> >
> > There is a general problem with the n-ary operators regarding type
> safety,
> > thats why we now have SingleInput and Co (two-input) operators. I think
> we
> > should keep these.
> >
> > On Fri, May 8, 2015 at 11:38 AM, Aljoscha Krettek <aljos...@apache.org>
> > wrote:
> >
> >> Hi,
> >> since I'm currently reworking the Stream operators I thought it's a
> >> good time to talk about the naming of some classes. We have some
> >> legacy problems with lots of Operators, OperatorBases, TwoInput,
> >> OneInput, Unary, Binary, etc. And maybe we can break things in
> >> streaming to have more consistent and future-proof naming.
> >>
> >> In streaming, there are:
> >> - Tasks, these are an AbstractInvokabe and contain the main loop of a
> >> streaming vertex. They read from the inputs and forward data to the
> >> operator implementation.
> >>
> >> - Operators, these are invoked by a Task and are responsible for the
> >> actual logic of the operator. Think Map, Join, Reduce and so on. These
> >> are responsible for calling the user-defined function.
> >>
> >> - Operators (again, I know), these are user facing classes (some
> >> derived from DataStream, some not). There is for example
> >> SingleOutputStreamOperator, for the result of a DataStream
> >> transformation that has a single output. There are also
> >> TemporalOperator and its derived classes StreamCrossOperator and
> >> StreamJoinOperator. The actual operator inside a task (the ones I
> >> mentioned before that are responsible for the user logic) that
> >> executes a temporal join is called CoStreamWindow (with a
> >> JoinWindowFunction).
> >>
> >> As I currently have it in my PR, there are two Task classes, one for
> >> single input, and one for two-input operators. There are also the
> >> corresponding operator interfaces for unary and binary operators (see
> >> what I did there ... :D).
> >>
> >> What should we call all these classes (concepts). Also I'm heavily in
> >> favour of dropping all the Stream (or Streaming) prefixes and suffixes
> >> from the class names. I know I'm in streaming because the package is
> >> named streaming. And we should not restrain ourselves because the
> >> batch API also has things called operator.
> >>
> >> Also, the concept of one-input, two-input tasks and operators is not
> >> very scalable, Maybe we should have a single interface for operators
> >> that has a receiveElement(int, element) method that tells the operator
> >> from which input an element came. Then we can scale this to n-ary
> >> operators. This would of course have the overhead of always sending
> >> along the number of the input instead of encoding the input number in
> >> the method name, such as receiveElement1() and receiveElement2().
> >>
> >> Any thoughts? :D (I know I'm writing the long annoying emails today
> >> but I think it is important we discuss these things before being stuck
> >> with them.)
> >>
> >> Cheers,
> >> Aljoscha
> >>
>

Reply via email to