I would deprecate "org.apache.flink.api.common.time.Time" only if we have alternative methods for the window api. If users can only write code by using a deprecated class, that would not be a good experience.
Otherwise sound good. Best, Stephan On Thu, Sep 12, 2019 at 11:23 AM Zili Chen <wander4...@gmail.com> wrote: > Hi, > > I've given it a try to switch to Java's Duration for all runtime Java code. > Generally, most of its usages are for @RpcTimeout and testing timeout. > > However, do a clean work without touch public interface possibly introduce > bridge codes convert runtime Java Duration to Flink's Time. I don't think > it is worth to do the detailed distinguish job, and even we possibly > introduce > extra bridge codes. > > Given this, I just filed an issue about this topic(we should have done :-)) > FLINK-14068[1] as a subtask of FLINK-3957 "Breaking changes for Flink 2.0". > > For what we can do now, > > 1. Deprecate org.apache.flink.api.common.time.Time also. > 2. Stop introduce more usages of Flink's Time, specifically for testing > timeout. This could be manually checked when review pull request(not > perfect > thought :\) > > We can do the final removal at once when prepare for 2.0 though. > > Best, > tison. > > [1]https://issues.apache.org/jira/browse/FLINK-14068 > > > Stephan Ewen <se...@apache.org> 于2019年8月27日周二 上午1:19写道: > > > Seems everyone is in favor in principle. > > > > - For public APIs, I would keep Time for now (to not break the API). > > Maybe add a Duration variant and deprecate the Time variant, but not > remove > > it before Flink 1.0 > > - For all runtime Java code, switch to Java's Duration now > > - For all Scala code let's see how much we can switch to Java Durations > > without blowing up stuff. After all, like Tison said, we want to get the > > runtime Scala free in the future. > > > > On Mon, Aug 26, 2019 at 3:45 AM Jark Wu <imj...@gmail.com> wrote: > > > > > +1 to use Java's Duration instead of Flink's Time. > > > > > > Regarding to the Duration parsing, we have mentioned this in FLIP-54[1] > > to > > > use `org.apache.flink.util.TimeUtils` for the parsing. > > > > > > Best, > > > Jark > > > > > > [1]: > > > > > > > > > https://docs.google.com/document/d/1IQ7nwXqmhCy900t2vQLEL3N2HIdMg-JO8vTzo1BtyKU/edit#heading=h.egdwkc93dn1k > > > > > > On Sat, 24 Aug 2019 at 18:24, Zhu Zhu <reed...@gmail.com> wrote: > > > > > > > +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 > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > > >