+1 since Java Duration is more common and powerful than Flink Time.

For whether to drop scala Duration for parsing duration OptionConfig, I
think it's another question and should be discussed in another thread.

Thanks,
Zhu Zhu

Becket Qin <becket....@gmail.com> 于2019年8月24日周六 下午4:16写道:

> +1, makes sense. BTW, we probably need a FLIP as this is a public API
> change.
>
> On Sat, Aug 24, 2019 at 8:11 AM SHI Xiaogang <shixiaoga...@gmail.com>
> wrote:
>
> > +1 to replace Flink's time with Java's Duration.
> >
> > Besides, i also suggest to use Java's Instant for "point-in-time".
> > It can take care of time units when we calculate Duration between
> different
> > instants.
> >
> > Regards,
> > Xiaogang
> >
> > Zili Chen <wander4...@gmail.com> 于2019年8月24日周六 上午10:45写道:
> >
> > > Hi vino,
> > >
> > > I agree that it introduces extra complexity to replace Duration(Scala)
> > > with Duration(Java) *in Scala code*. We could separate the usage for
> each
> > > language and use a bridge when necessary.
> > >
> > > As a matter of fact, Scala concurrent APIs(including Duration) are used
> > > more than necessary at least in flink-runtime. Also we even try to make
> > > flink-runtime scala free.
> > >
> > > Best,
> > > tison.
> > >
> > >
> > > vino yang <yanghua1...@gmail.com> 于2019年8月24日周六 上午10:05写道:
> > >
> > > > +1 to replace the Time class provided by Flink with Java's Duration:
> > > >
> > > >
> > > >    - Java's Duration has better representation than the Flink's Time
> > > class;
> > > >    - As a built-in Java class, Duration class has a clear advantage
> > over
> > > >    Java's Time class when interacting with other Java APIs and
> > > third-party
> > > >    libraries;
> > > >
> > > >
> > > > But I have reservations about replacing the Duration and FineDuration
> > > > classes in scala with the Duration class in Java. Java and Scala have
> > > > different types of systems. Currently, Duration (scala) and
> > FineDuration
> > > > (scala) work well.  In addition, this work brings additional
> complexity
> > > and
> > > > cost compared to the gains obtained.
> > > >
> > > > Best,
> > > > Vino
> > > >
> > > > Zili Chen <wander4...@gmail.com> 于2019年8月23日周五 下午11:14写道:
> > > >
> > > > > Hi Stephan,
> > > > >
> > > > > I like the idea unify usage of time/duration api. We actually
> > > > > use at least five different classes for this purposes(see below).
> > > > >
> > > > > One thing I'd like to pick up is that duration configuration
> > > > > in Flink is almost in pattern as "60 s" that fits in the pattern
> > > > > parsed by scala.concurrent.duration.Duration. AFAIK Duration
> > > > > in Java 8 doesn't support this pattern. However, we can solve
> > > > > it by introduce a DurationUtils.
> > > > >
> > > > > Also to clarify, we now have (correct me if any other)
> > > > >
> > > > > java.time.Duration
> > > > > scala.concurrent.duration.Duration
> > > > > scala.concurrent.duration.FiniteDuration
> > > > > org.apache.flink.api.common.time.Time
> > > > > org.apache.flink.streaming.api.windowing.time.Time
> > > > >
> > > > > in use. If we'd prefer java.time.Duration, it is worth to consider
> > > > > whether we unify all of them into Java's Duration, i.e., Java's
> > > > > Duration is the first class time/duration api, while others should
> > > > > be converted into or out from it.
> > > > >
> > > > > Best,
> > > > > tison.
> > > > >
> > > > >
> > > > > Stephan Ewen <se...@apache.org> 于2019年8月23日周五 下午10:45写道:
> > > > >
> > > > > > Hi all!
> > > > > >
> > > > > > Many parts of the code use Flink's "Time" class. The Time really
> > is a
> > > > > "time
> > > > > > interval" or a "Duration".
> > > > > >
> > > > > > Since Java 8, there is a Java class "Duration" that is nice and
> > > > flexible
> > > > > to
> > > > > > use.
> > > > > > I would suggest we start using Java Duration instead and drop
> Time
> > as
> > > > > much
> > > > > > as possible in the runtime from now on.
> > > > > >
> > > > > > Maybe even drop that class from the API in Flink 2.0.
> > > > > >
> > > > > > Best,
> > > > > > Stephan
> > > > > >
> > > > >
> > > >
> > >
> >
>

Reply via email to