+1 to Till's points on #2 and #5, especially the potential non-disruptive, gradual migration approach if we decide to go that route.
To add on, I want to point it out that we can actually start with flink-shaded project [1] which is a perfect candidate for PoC. It's of much smaller size, totally isolated from and not interfered with flink project [2], and it actually covers most of our practical feature requirements for a build tool - all making it an ideal experimental field. [1] https://github.com/apache/flink-shaded [2] https://github.com/apache/flink On Fri, Aug 16, 2019 at 4:52 AM Till Rohrmann <trohrm...@apache.org> wrote: > For the sake of keeping the discussion focused and not cluttering the > discussion thread I would suggest to split the detailed reporting for > reusing JVMs to a separate thread and cross linking it from here. > > Cheers, > Till > > On Fri, Aug 16, 2019 at 1:36 PM Chesnay Schepler <ches...@apache.org> > wrote: > > > Update: > > > > TL;DR: table-planner is a good candidate for enabling fork reuse right > > away, while flink-tests has the potential for huge savings, but we have > > to figure out some issues first. > > > > > > Build link: https://travis-ci.org/zentol/flink/builds/572659220 > > > > 4/8 profiles failed. > > > > No speedup in libraries, python, blink_planner, 7 minutes saved in > > libraries (table-planner). > > > > The kafka and connectors profiles both fail in kafka tests due to > > producer leaks, and no speed up could be confirmed so far: > > > > java.lang.AssertionError: Detected producer leak. Thread name: > > kafka-producer-network-thread | producer-239 > > at org.junit.Assert.fail(Assert.java:88) > > at > > > org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011ITCase.checkProducerLeak(FlinkKafkaProducer011ITCase.java:677) > > at > > > org.apache.flink.streaming.connectors.kafka.FlinkKafkaProducer011ITCase.testFlinkKafkaProducer011FailBeforeNotify(FlinkKafkaProducer011ITCase.java:210) > > > > > > The tests profile failed due to various errors in migration tests: > > > > junit.framework.AssertionFailedError: Did not see the expected > accumulator > > results within time limit. > > at > > > org.apache.flink.test.migration.TypeSerializerSnapshotMigrationITCase.testSavepoint(TypeSerializerSnapshotMigrationITCase.java:141) > > > > *However*, a normal tests run takes 40 minutes, while this one above > > failed after 19 minutes and is only missing the migration tests (which > > currently need 6-7 minutes). So we could save somewhere between 15 to 20 > > minutes here. > > > > > > Finally, the misc profiles fails in YARN: > > > > java.lang.AssertionError > > at org.apache.flink.yarn.YARNITCase.setup(YARNITCase.java:64) > > > > No significant speedup could be observed in other modules; for > > flink-yarn-tests we can maybe get a minute or 2 out of it. > > > > On 16/08/2019 10:43, Chesnay Schepler wrote: > > > There appears to be a general agreement that 1) should be looked into; > > > I've setup a branch with fork reuse being enabled for all tests; will > > > report back the results. > > > > > > On 15/08/2019 09:38, Chesnay Schepler wrote: > > >> Hello everyone, > > >> > > >> improving our build times is a hot topic at the moment so let's > > >> discuss the different ways how they could be reduced. > > >> > > >> > > >> Current state: > > >> > > >> First up, let's look at some numbers: > > >> > > >> 1 full build currently consumes 5h of build time total ("total > > >> time"), and in the ideal case takes about 1h20m ("run time") to > > >> complete from start to finish. The run time may fluctuate of course > > >> depending on the current Travis load. This applies both to builds on > > >> the Apache and flink-ci Travis. > > >> > > >> At the time of writing, the current queue time for PR jobs (reminder: > > >> running on flink-ci) is about 30 minutes (which basically means that > > >> we are processing builds at the rate that they come in), however we > > >> are in an admittedly quiet period right now. > > >> 2 weeks ago the queue times on flink-ci peaked at around 5-6h as > > >> everyone was scrambling to get their changes merged in time for the > > >> feature freeze. > > >> > > >> (Note: Recently optimizations where added to ci-bot where pending > > >> builds are canceled if a new commit was pushed to the PR or the PR > > >> was closed, which should prove especially useful during the rush > > >> hours we see before feature-freezes.) > > >> > > >> > > >> Past approaches > > >> > > >> Over the years we have done rather few things to improve this > > >> situation (hence our current predicament). > > >> > > >> Beyond the sporadic speedup of some tests, the only notable reduction > > >> in total build times was the introduction of cron jobs, which > > >> consolidated the per-commit matrix from 4 configurations (different > > >> scala/hadoop versions) to 1. > > >> > > >> The separation into multiple build profiles was only a work-around > > >> for the 50m limit on Travis. Running tests in parallel has the > > >> obvious potential of reducing run time, but we're currently hitting a > > >> hard limit since a few modules (flink-tests, flink-runtime, > > >> flink-table-planner-blink) are so loaded with tests that they nearly > > >> consume an entire profile by themselves (and thus no further > > >> splitting is possible). > > >> > > >> The rework that introduced stages, at the time of introduction, did > > >> also not provide a speed up, although this changed slightly once more > > >> profiles were added and some optimizations to the caching have been > > >> made. > > >> > > >> Very recently we modified the surefire-plugin configuration for > > >> flink-table-planner-blink to reuse JVM forks for IT cases, providing > > >> a significant speedup (18 minutes!). So far we have not seen any > > >> negative consequences. > > >> > > >> > > >> Suggestions > > >> > > >> This is a list of /all /suggestions for reducing run/total times that > > >> I have seen recently (in other words, they aren't necessarily mine > > >> nor may I agree with all of them). > > >> > > >> 1. Enable JVM reuse for IT cases in more modules. > > >> * We've seen significant speedups in the blink planner, and this > > >> should be applicable for all modules. However, I presume > there's > > >> a reason why we disabled JVM reuse (information on this would > be > > >> appreciated) > > >> 2. Custom differential build scripts > > >> * Setup custom scripts for determining which modules might be > > >> affected by change, and manipulate the splits accordingly. This > > >> approach is conceptually quite straight-forward, but has limits > > >> since it has to be pessimistic; i.e. a change in flink-core > > >> _must_ result in testing all modules. > > >> 3. Only run smoke tests when PR is opened, run heavy tests on demand. > > >> * With the introduction of the ci-bot we now have significantly > > >> more options on how to handle PR builds. One option could be to > > >> only run basic tests when the PR is created (which may be only > > >> modified modules, or all unit tests, or another low-cost > > >> scheme), and then have a committer trigger other builds (full > > >> test run, e2e tests, etc...) on demand. > > >> 4. Move more tests into cron builds > > >> * The budget version of 3); move certain tests that are either > > >> expensive (like some runtime tests that take minutes) or in > > >> rarely modified modules (like gelly) into cron jobs. > > >> 5. Gradle > > >> * Gradle was brought up a few times for it's built-in support for > > >> differential builds; basically providing 2) without the > overhead > > >> of maintaining additional scripts. > > >> * To date no PoC was provided that shows it working in our CI > > >> environment (i.e., handling splits & caching etc). > > >> * This is the most disruptive change by a fair margin, as it > would > > >> affect the entire project, developers and potentially users (f > > >> they build from source). > > >> 6. CI service > > >> * Our current artifact caching setup on Travis is basically a > > >> hack; we're basically abusing the Travis cache, which is meant > > >> for long-term caching, to ship build artifacts across jobs. > It's > > >> brittle at times due to timing/visibility issues and on > branches > > >> the cleanup processes can interfere with running builds. It is > > >> also not as effective as it could be. > > >> * There are CI services that provide build artifact caching out > of > > >> the box, which could be useful for us. > > >> * To date, no PoC for using another CI service has been provided. > > >> > > >> > > > > > > > > > > >