You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Udi Meiri <eh...@google.com> on 2018/06/06 02:24:25 UTC

Re: Proposal: keeping precommit times fast

I've been having a separate discussion on the proposal doc, which is ready
for another round of reviews.
Change summary:
- Changed fast requirement to be < 30 minutes and simplify the check as an
aggregate for each precommit job type.
- Updated slowness notification methods to include automated methods: as a
precommit check result type on GitHub, as a bug.
- Merged in the metrics design doc.
- Added detailed design section.
- Added list of deliverables.

What I would like is consensus regarding:
- How fast we want precommit runs to be. I propose 30m.
- Deadline for fixing a slow test before it is temporarily removed from
precommit. I propose 24 hours.


Replying to the thread:

1. I like the idea of using the Jenkins Job Cacher Plugin to skip
unaffected tests (BEAM-4400).

2. Java Precommit tests include integration tests (example
<https://builds.apache.org/view/A-D/view/Beam/job/beam_PreCommit_Java_GradleBuild/lastCompletedBuild/testReport/org.apache.beam.examples/>
).
We could split these out to get much faster results, i.e., a separate
precommit just for basic integration tests (which will still need to run in
<30m).
Perhaps lint checks for Python could be split out as well.

I'll add these suggestions to the doc tomorrow.

On Thu, May 24, 2018 at 9:25 AM Scott Wegner <sw...@google.com> wrote:

> So, it sounds like there's agreement that we should improve precommit
> times by only running necessary tests, and configuring Jenkins Job
> Caching + Gradle build cache is a path to get there. I've filed BEAM-4400
> [1] to follow-up on this.
>
> Getting back to Udi's original proposal [2]: I see value in defining a
> metric and target for overall pre-commit timing. The proposal for an
> initial "2 hour" target is helpful as a guardrail: we're already hitting
> it, but if we drift to a point where we're not, that should trigger some
> action to be taken to get back to a healthy state.
>
> I wouldn't mind separately setting a more aspiration goal of getting the
> pre-commits even faster (i.e. 15-30 mins), but I suspect that would require
> a concerted effort to evaluate and improve existing tests across the
> codebase. One idea would be to set up ensure the metric reporting can show
> the trend, and which tests are responsible for the most walltime, so that
> we know where to invest any efforts to improve tests.
>
>
> [1] https://issues.apache.org/jira/browse/BEAM-4400
> [2]
> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>
>
> On Wed, May 23, 2018 at 11:46 AM Kenneth Knowles <kl...@google.com> wrote:
>
>> With regard to the Job Cacher Plugin: I think it is an infra ticket to
>> install? And I guess we need it longer term when we move to containerized
>> builds anyhow? One thing I've experienced with the Travis-CI cache is that
>> the time spent uploading & downloading the remote cache - in that case of
>> all the pip installed dependencies - negated the benefits. Probably for
>> Beam it will have a greater benefit if we can skip most of the build.
>>
>> Regarding integration tests in precommit: I think it is OK to run maybe
>> one Dataflow job in precommit, but it should be in parallel with the unit
>> tests and just a smoke test that takes 5 minutes, not a suite that takes 35
>> minutes. So IMO that is low-hanging fruit. If this would make postcommit
>> unstable, then it also means precommit is unstable. Both are troublesome.
>>
>> More short term, some possible hacks:
>>
>>  - Point gradle to cache outside the git workspace. We already did this
>> for .m2 and it helped a lot.
>>  - Intersect touched files with projects. Our nonstandard project names
>> might be a pain here. Not sure if fixing that is on the roadmap.
>>
>> Kenn
>>
>> On Wed, May 23, 2018 at 9:31 AM Ismaël Mejía <ie...@gmail.com> wrote:
>>
>>> I second Robert idea of ‘inteligently’ running only the affected tests,
>>> probably
>>> there is no need to run Java for a go fix (and eventually if any issue it
>>> can be
>>> catched in postcommit), same for a dev who just fixed something in
>>> KafkaIO
>>> and has
>>> to wait for other IO tests to pass. I suppose that languages, IOs and
>>> extensions
>>> are ‘easy’ to isolate so maybe we can start with those.
>>>
>>> Earlier signals are also definitely great to have too, but not sure how
>>> we
>>> can
>>> have those with the current infra.
>>>
>>>  From a quicklook the biggest time is consumed by the examples module
>>> probably
>>> because they run in Dataflow with real IOs no?, that module alone takes
>>> ~35
>>> minutes, so maybe moving it to postcommit will gain us some quick
>>> improvement.
>>> On the other hand we should probably not dismiss the consequences of
>>> moving
>>> more
>>> stuff to postcommit given that our current postcommit is not the most
>>> stable, or
>>> the quickest, only the Dataflow suite takes 1h30!
>>>
>>>
>>> On Tue, May 22, 2018 at 12:01 AM Mikhail Gryzykhin <mi...@google.com>
>>> wrote:
>>>
>>> > What we can do here is estimate how much effort we want to put in and
>>> set
>>> remote target.
>>> > Such as:
>>> > Third quarter 2018 -- 1hr SLO
>>> > Forth quarter 2018 -- 30min SLO,
>>> > etc.
>>>
>>> > Combined with policy for newly added tests, this can give us some goal
>>> to
>>> aim for.
>>>
>>> > --Mikhail
>>>
>>> > Have feedback?
>>>
>>>
>>> > On Mon, May 21, 2018 at 2:06 PM Scott Wegner <sw...@google.com>
>>> wrote:
>>>
>>> >> Thanks for the proposal, I left comments in the doc. Overall I think
>>> it's a great idea.
>>>
>>> >> I've seen other projects with much faster pre-commits, and it requires
>>> strict guidelines on unit test design and keeping tests isolated
>>> in-memory
>>> as much as possible. That's not currently the case in Java; we have
>>> pre-commits which submit pipelines to Dataflow service.
>>>
>>> >> I don't know if it's feasible to get Java down to 15-20 mins in the
>>> short term, but a good starting point would be to document the
>>> requirements
>>> for a test to run as pre-commit, and start enforcing it for new tests.
>>>
>>>
>>> >> On Fri, May 18, 2018 at 3:25 PM Henning Rohde <he...@google.com>
>>> wrote:
>>>
>>> >>> Good proposal. I think it should be considered in tandem with the "No
>>> commit on red post-commit" proposal and could be far more ambitious than
>>> 2
>>> hours. For example, something in the <15-20 mins range, say, would be
>>> much
>>> less of an inconvenience to the development effort. Go takes ~3 mins,
>>> which
>>> means that it is practical to wait until a PR is green before asking
>>> anyone
>>> to look at it. If I need to wait for a Java or Python pre-commit, I task
>>> switch and come back later. If the post-commits are enforced to be green,
>>> we could possibly gain a much more productive flow at the cost of the
>>> occasional post-commit break, compared to now. Maybe IOs can be less
>>> extensively tested pre-commit, for example, or only if actually changed?
>>>
>>> >>> I also like Robert's suggestion of spitting up pre-commits into
>>> something more fine-grained to get a clear partial signal quicker. If we
>>> have an adequate number of Jenkins slots, it might also speed things up
>>> overall.
>>>
>>> >>> Thanks,
>>> >>>    Henning
>>>
>>> >>> On Fri, May 18, 2018 at 12:30 PM Scott Wegner <sw...@google.com>
>>> wrote:
>>>
>>> >>>> re: intelligently skipping tests for code that doesn't change (i.e.
>>> Java tests on Python PR): this should be possible. We already have
>>> build-caching enabled in Gradle, but I believe it is local to the git
>>> workspace and doesn't persist between Jenkins runs.
>>>
>>> >>>> With a quick search, I see there is a Jenkins Build Cacher Plugin
>>> [1]
>>> that hooks into Gradle build cache and does exactly what we need. Does
>>> anybody know whether we could get this enabled on our Jenkins?
>>>
>>> >>>> [1] https://wiki.jenkins.io/display/JENKINS/Job+Cacher+Plugin
>>>
>>> >>>> On Fri, May 18, 2018 at 12:08 PM Robert Bradshaw <
>>> robertwb@google.com>
>>> wrote:
>>>
>>> >>>>> [somehow  my email got garbled...]
>>>
>>> >>>>> Now that we're using gradle, perhaps we could be more intelligent
>>> about only running the affected tests? E.g. when you touch Python (or Go)
>>> you shouldn't need to run the Java precommit at all, which would reduce
>>> the
>>> latency for those PRs and also the time spent in queue. Presumably this
>>> could even be applied per-module for the Java tests. (Maybe a large,
>>> shared
>>> build cache could help here as well...)
>>>
>>> >>>>> I also wouldn't be opposed to a quicker immediate signal, plus more
>>> extensive tests before actually merging. It's also nice to not have to
>>> wait
>>> an hour to see that you have a lint error; quick stuff like that could be
>>> signaled quickly before a contributor looses context.
>>>
>>> >>>>> - Robert
>>>
>>>
>>>
>>> >>>>> On Fri, May 18, 2018 at 5:55 AM Kenneth Knowles <kl...@google.com>
>>> wrote:
>>>
>>> >>>>>> I like the idea. I think it is a good time for the project to
>>> start
>>> tracking this and keeping it usable.
>>>
>>> >>>>>> Certainly 2 hours is more than enough, is that not so? The Java
>>> precommit seems to take <=40 minutes while Python takes ~20 and Go is so
>>> fast it doesn't matter. Do we have enough stragglers that we don't make
>>> it
>>> in the 95th percentile? Is the time spent in the Jenkins queue?
>>>
>>> >>>>>> For our current coverage, I'd be willing to go for:
>>>
>>> >>>>>>    - 1 hr hard cap (someone better at stats could choose %ile)
>>> >>>>>>    - roll back or remove test from precommit if fix looks like
>>> more
>>> than 1 week (roll back if it is perf degradation, remove test from
>>> precommit if it is additional coverage that just doesn't fit in the time)
>>>
>>> >>>>>> There's a longer-term issue that doing a full build each time is
>>> expected to linearly scale up with the size of our repo (it is the
>>> monorepo
>>> problem but for a minirepo) so there is no cap that is feasible until we
>>> have effective cross-build caching. And my long-term goal would be <30
>>> minutes. At the latency of opening a pull request and then checking your
>>> email that's not burdensome, but an hour is.
>>>
>>> >>>>>> Kenn
>>>
>>> >>>>>> On Thu, May 17, 2018 at 6:54 PM Udi Meiri <eh...@google.com>
>>> wrote:
>>>
>>> >>>>>>> HI,
>>> >>>>>>> I have a proposal to improve contributor experience by keeping
>>> precommit times low.
>>>
>>> >>>>>>> I'm looking to get community consensus and approval about:
>>> >>>>>>> 1. How long should precommits take. 2 hours @95th percentile over
>>> the past 4 weeks is the current proposal.
>>> >>>>>>> 2. The process for dealing with slowness. Do we: fix, roll back,
>>> remove a test from precommit?
>>> >>>>>>> Rolling back if a fix is estimated to take longer than 2 weeks is
>>> the current proposal.
>>>
>>>
>>>
>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>
>>

Re: Proposal: keeping precommit times fast

Posted by Robert Bradshaw <ro...@google.com>.
No, this isn't the kind of thing that should require a vote (unless someone
really wants a vote).

On Thu, Jun 7, 2018 at 9:29 AM Udi Meiri <eh...@google.com> wrote:

> Would I need a vote on installing this plugin, or can I just open a ticket
> to infra?
>
> On Wed, Jun 6, 2018, 16:18 Robert Bradshaw <ro...@google.com> wrote:
>
>> Even if it's not perfect, seems like it'd surely be a net win (and
>> probably a large one). Also, the build cache should look back at more than
>> just the single previous build, so if any previous jobs (up to the cache
>> size limit) built/tested artifacts unchanged by the current PR, the results
>> would live in the cache.
>>
>> I would look at (a) and (b) only if this isn't already good enough.
>>
>> On Wed, Jun 6, 2018 at 3:50 PM Udi Meiri <eh...@google.com> wrote:
>>
>>> To follow up on the Jenkins Job Cacher Plugin:
>>>
>>> Using a Jenkins plugin to save and reuse the Gradle cache for successive
>>> precommit jobs.
>>> The problem with this approach is that the precommit runs that a Jenkins
>>> server runs are unrelated.
>>> Say you have 2 PRs, A and B, and the precommit job for B reuses the
>>> cache left by the job for A.
>>> The diff between the two will cause tests affected both by A and B to be
>>> rerun (at least).
>>> If A modifies Python code, then the job for B must rerun ALL Python
>>> tests (since Gradle doesn't do dependency tracking for Python).
>>>
>>> Proposal:
>>> a. The cache plugin is still useful for successive Java precommit jobs,
>>> but not for Python. (Go, I have no idea)
>>> We could use it exclusively for Java precommits.
>>> b. To avoid running precommit jobs for code not touched by a PR, look at
>>> the paths of files changed.
>>> For example, a PR touching only files under sdks/python/... need only
>>> run Python precommit tests.
>>>
>>> On Tue, Jun 5, 2018 at 7:24 PM Udi Meiri <eh...@google.com> wrote:
>>>
>>>> I've been having a separate discussion on the proposal doc, which is
>>>> ready for another round of reviews.
>>>> Change summary:
>>>> - Changed fast requirement to be < 30 minutes and simplify the check as
>>>> an aggregate for each precommit job type.
>>>> - Updated slowness notification methods to include automated methods:
>>>> as a precommit check result type on GitHub, as a bug.
>>>> - Merged in the metrics design doc.
>>>> - Added detailed design section.
>>>> - Added list of deliverables.
>>>>
>>>> What I would like is consensus regarding:
>>>> - How fast we want precommit runs to be. I propose 30m.
>>>> - Deadline for fixing a slow test before it is temporarily removed from
>>>> precommit. I propose 24 hours.
>>>>
>>>>
>>>> Replying to the thread:
>>>>
>>>> 1. I like the idea of using the Jenkins Job Cacher Plugin to skip
>>>> unaffected tests (BEAM-4400).
>>>>
>>>> 2. Java Precommit tests include integration tests (example
>>>> <https://builds.apache.org/view/A-D/view/Beam/job/beam_PreCommit_Java_GradleBuild/lastCompletedBuild/testReport/org.apache.beam.examples/>
>>>> ).
>>>> We could split these out to get much faster results, i.e., a separate
>>>> precommit just for basic integration tests (which will still need to run in
>>>> <30m).
>>>> Perhaps lint checks for Python could be split out as well.
>>>>
>>>> I'll add these suggestions to the doc tomorrow.
>>>>
>>>> On Thu, May 24, 2018 at 9:25 AM Scott Wegner <sw...@google.com>
>>>> wrote:
>>>>
>>>>> So, it sounds like there's agreement that we should improve precommit
>>>>> times by only running necessary tests, and configuring Jenkins Job
>>>>> Caching + Gradle build cache is a path to get there. I've filed BEAM-4400
>>>>> [1] to follow-up on this.
>>>>>
>>>>> Getting back to Udi's original proposal [2]: I see value in defining a
>>>>> metric and target for overall pre-commit timing. The proposal for an
>>>>> initial "2 hour" target is helpful as a guardrail: we're already hitting
>>>>> it, but if we drift to a point where we're not, that should trigger some
>>>>> action to be taken to get back to a healthy state.
>>>>>
>>>>> I wouldn't mind separately setting a more aspiration goal of getting
>>>>> the pre-commits even faster (i.e. 15-30 mins), but I suspect that would
>>>>> require a concerted effort to evaluate and improve existing tests across
>>>>> the codebase. One idea would be to set up ensure the metric reporting can
>>>>> show the trend, and which tests are responsible for the most walltime, so
>>>>> that we know where to invest any efforts to improve tests.
>>>>>
>>>>>
>>>>> [1] https://issues.apache.org/jira/browse/BEAM-4400
>>>>> [2]
>>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>>
>>>>>
>>>>> On Wed, May 23, 2018 at 11:46 AM Kenneth Knowles <kl...@google.com>
>>>>> wrote:
>>>>>
>>>>>> With regard to the Job Cacher Plugin: I think it is an infra ticket
>>>>>> to install? And I guess we need it longer term when we move to
>>>>>> containerized builds anyhow? One thing I've experienced with the Travis-CI
>>>>>> cache is that the time spent uploading & downloading the remote cache - in
>>>>>> that case of all the pip installed dependencies - negated the benefits.
>>>>>> Probably for Beam it will have a greater benefit if we can skip most of the
>>>>>> build.
>>>>>>
>>>>>> Regarding integration tests in precommit: I think it is OK to run
>>>>>> maybe one Dataflow job in precommit, but it should be in parallel with the
>>>>>> unit tests and just a smoke test that takes 5 minutes, not a suite that
>>>>>> takes 35 minutes. So IMO that is low-hanging fruit. If this would make
>>>>>> postcommit unstable, then it also means precommit is unstable. Both are
>>>>>> troublesome.
>>>>>>
>>>>>> More short term, some possible hacks:
>>>>>>
>>>>>>  - Point gradle to cache outside the git workspace. We already did
>>>>>> this for .m2 and it helped a lot.
>>>>>>  - Intersect touched files with projects. Our nonstandard project
>>>>>> names might be a pain here. Not sure if fixing that is on the roadmap.
>>>>>>
>>>>>> Kenn
>>>>>>
>>>>>> On Wed, May 23, 2018 at 9:31 AM Ismaël Mejía <ie...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I second Robert idea of ‘inteligently’ running only the affected
>>>>>>> tests,
>>>>>>> probably
>>>>>>> there is no need to run Java for a go fix (and eventually if any
>>>>>>> issue it
>>>>>>> can be
>>>>>>> catched in postcommit), same for a dev who just fixed something in
>>>>>>> KafkaIO
>>>>>>> and has
>>>>>>> to wait for other IO tests to pass. I suppose that languages, IOs and
>>>>>>> extensions
>>>>>>> are ‘easy’ to isolate so maybe we can start with those.
>>>>>>>
>>>>>>> Earlier signals are also definitely great to have too, but not sure
>>>>>>> how we
>>>>>>> can
>>>>>>> have those with the current infra.
>>>>>>>
>>>>>>>  From a quicklook the biggest time is consumed by the examples module
>>>>>>> probably
>>>>>>> because they run in Dataflow with real IOs no?, that module alone
>>>>>>> takes ~35
>>>>>>> minutes, so maybe moving it to postcommit will gain us some quick
>>>>>>> improvement.
>>>>>>> On the other hand we should probably not dismiss the consequences of
>>>>>>> moving
>>>>>>> more
>>>>>>> stuff to postcommit given that our current postcommit is not the most
>>>>>>> stable, or
>>>>>>> the quickest, only the Dataflow suite takes 1h30!
>>>>>>>
>>>>>>>
>>>>>>> On Tue, May 22, 2018 at 12:01 AM Mikhail Gryzykhin <
>>>>>>> migryz@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> > What we can do here is estimate how much effort we want to put in
>>>>>>> and set
>>>>>>> remote target.
>>>>>>> > Such as:
>>>>>>> > Third quarter 2018 -- 1hr SLO
>>>>>>> > Forth quarter 2018 -- 30min SLO,
>>>>>>> > etc.
>>>>>>>
>>>>>>> > Combined with policy for newly added tests, this can give us some
>>>>>>> goal to
>>>>>>> aim for.
>>>>>>>
>>>>>>> > --Mikhail
>>>>>>>
>>>>>>> > Have feedback?
>>>>>>>
>>>>>>>
>>>>>>> > On Mon, May 21, 2018 at 2:06 PM Scott Wegner <sw...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >> Thanks for the proposal, I left comments in the doc. Overall I
>>>>>>> think
>>>>>>> it's a great idea.
>>>>>>>
>>>>>>> >> I've seen other projects with much faster pre-commits, and it
>>>>>>> requires
>>>>>>> strict guidelines on unit test design and keeping tests isolated
>>>>>>> in-memory
>>>>>>> as much as possible. That's not currently the case in Java; we have
>>>>>>> pre-commits which submit pipelines to Dataflow service.
>>>>>>>
>>>>>>> >> I don't know if it's feasible to get Java down to 15-20 mins in
>>>>>>> the
>>>>>>> short term, but a good starting point would be to document the
>>>>>>> requirements
>>>>>>> for a test to run as pre-commit, and start enforcing it for new
>>>>>>> tests.
>>>>>>>
>>>>>>>
>>>>>>> >> On Fri, May 18, 2018 at 3:25 PM Henning Rohde <he...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>> Good proposal. I think it should be considered in tandem with
>>>>>>> the "No
>>>>>>> commit on red post-commit" proposal and could be far more ambitious
>>>>>>> than 2
>>>>>>> hours. For example, something in the <15-20 mins range, say, would
>>>>>>> be much
>>>>>>> less of an inconvenience to the development effort. Go takes ~3
>>>>>>> mins, which
>>>>>>> means that it is practical to wait until a PR is green before asking
>>>>>>> anyone
>>>>>>> to look at it. If I need to wait for a Java or Python pre-commit, I
>>>>>>> task
>>>>>>> switch and come back later. If the post-commits are enforced to be
>>>>>>> green,
>>>>>>> we could possibly gain a much more productive flow at the cost of the
>>>>>>> occasional post-commit break, compared to now. Maybe IOs can be less
>>>>>>> extensively tested pre-commit, for example, or only if actually
>>>>>>> changed?
>>>>>>>
>>>>>>> >>> I also like Robert's suggestion of spitting up pre-commits into
>>>>>>> something more fine-grained to get a clear partial signal quicker.
>>>>>>> If we
>>>>>>> have an adequate number of Jenkins slots, it might also speed things
>>>>>>> up
>>>>>>> overall.
>>>>>>>
>>>>>>> >>> Thanks,
>>>>>>> >>>    Henning
>>>>>>>
>>>>>>> >>> On Fri, May 18, 2018 at 12:30 PM Scott Wegner <
>>>>>>> swegner@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>> re: intelligently skipping tests for code that doesn't change
>>>>>>> (i.e.
>>>>>>> Java tests on Python PR): this should be possible. We already have
>>>>>>> build-caching enabled in Gradle, but I believe it is local to the git
>>>>>>> workspace and doesn't persist between Jenkins runs.
>>>>>>>
>>>>>>> >>>> With a quick search, I see there is a Jenkins Build Cacher
>>>>>>> Plugin [1]
>>>>>>> that hooks into Gradle build cache and does exactly what we need.
>>>>>>> Does
>>>>>>> anybody know whether we could get this enabled on our Jenkins?
>>>>>>>
>>>>>>> >>>> [1] https://wiki.jenkins.io/display/JENKINS/Job+Cacher+Plugin
>>>>>>>
>>>>>>> >>>> On Fri, May 18, 2018 at 12:08 PM Robert Bradshaw <
>>>>>>> robertwb@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>>> [somehow  my email got garbled...]
>>>>>>>
>>>>>>> >>>>> Now that we're using gradle, perhaps we could be more
>>>>>>> intelligent
>>>>>>> about only running the affected tests? E.g. when you touch Python
>>>>>>> (or Go)
>>>>>>> you shouldn't need to run the Java precommit at all, which would
>>>>>>> reduce the
>>>>>>> latency for those PRs and also the time spent in queue. Presumably
>>>>>>> this
>>>>>>> could even be applied per-module for the Java tests. (Maybe a large,
>>>>>>> shared
>>>>>>> build cache could help here as well...)
>>>>>>>
>>>>>>> >>>>> I also wouldn't be opposed to a quicker immediate signal, plus
>>>>>>> more
>>>>>>> extensive tests before actually merging. It's also nice to not have
>>>>>>> to wait
>>>>>>> an hour to see that you have a lint error; quick stuff like that
>>>>>>> could be
>>>>>>> signaled quickly before a contributor looses context.
>>>>>>>
>>>>>>> >>>>> - Robert
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> >>>>> On Fri, May 18, 2018 at 5:55 AM Kenneth Knowles <
>>>>>>> klk@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>>>> I like the idea. I think it is a good time for the project to
>>>>>>> start
>>>>>>> tracking this and keeping it usable.
>>>>>>>
>>>>>>> >>>>>> Certainly 2 hours is more than enough, is that not so? The
>>>>>>> Java
>>>>>>> precommit seems to take <=40 minutes while Python takes ~20 and Go
>>>>>>> is so
>>>>>>> fast it doesn't matter. Do we have enough stragglers that we don't
>>>>>>> make it
>>>>>>> in the 95th percentile? Is the time spent in the Jenkins queue?
>>>>>>>
>>>>>>> >>>>>> For our current coverage, I'd be willing to go for:
>>>>>>>
>>>>>>> >>>>>>    - 1 hr hard cap (someone better at stats could choose %ile)
>>>>>>> >>>>>>    - roll back or remove test from precommit if fix looks
>>>>>>> like more
>>>>>>> than 1 week (roll back if it is perf degradation, remove test from
>>>>>>> precommit if it is additional coverage that just doesn't fit in the
>>>>>>> time)
>>>>>>>
>>>>>>> >>>>>> There's a longer-term issue that doing a full build each time
>>>>>>> is
>>>>>>> expected to linearly scale up with the size of our repo (it is the
>>>>>>> monorepo
>>>>>>> problem but for a minirepo) so there is no cap that is feasible
>>>>>>> until we
>>>>>>> have effective cross-build caching. And my long-term goal would be
>>>>>>> <30
>>>>>>> minutes. At the latency of opening a pull request and then checking
>>>>>>> your
>>>>>>> email that's not burdensome, but an hour is.
>>>>>>>
>>>>>>> >>>>>> Kenn
>>>>>>>
>>>>>>> >>>>>> On Thu, May 17, 2018 at 6:54 PM Udi Meiri <eh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>>>>> HI,
>>>>>>> >>>>>>> I have a proposal to improve contributor experience by
>>>>>>> keeping
>>>>>>> precommit times low.
>>>>>>>
>>>>>>> >>>>>>> I'm looking to get community consensus and approval about:
>>>>>>> >>>>>>> 1. How long should precommits take. 2 hours @95th percentile
>>>>>>> over
>>>>>>> the past 4 weeks is the current proposal.
>>>>>>> >>>>>>> 2. The process for dealing with slowness. Do we: fix, roll
>>>>>>> back,
>>>>>>> remove a test from precommit?
>>>>>>> >>>>>>> Rolling back if a fix is estimated to take longer than 2
>>>>>>> weeks is
>>>>>>> the current proposal.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>>>>
>>>>>>

Re: Proposal: keeping precommit times fast

Posted by Scott Wegner <sw...@google.com>.
I don't see any downside to enabling this plugin; we previously
filed BEAM-4400 [1] for this work. I believe the next steps would be
working with Infra on enabling it.

[1] https://issues.apache.org/jira/browse/BEAM-4400

On Thu, Jun 7, 2018 at 9:29 AM Udi Meiri <eh...@google.com> wrote:

> Would I need a vote on installing this plugin, or can I just open a ticket
> to infra?
>
> On Wed, Jun 6, 2018, 16:18 Robert Bradshaw <ro...@google.com> wrote:
>
>> Even if it's not perfect, seems like it'd surely be a net win (and
>> probably a large one). Also, the build cache should look back at more than
>> just the single previous build, so if any previous jobs (up to the cache
>> size limit) built/tested artifacts unchanged by the current PR, the results
>> would live in the cache.
>>
>> I would look at (a) and (b) only if this isn't already good enough.
>>
>> On Wed, Jun 6, 2018 at 3:50 PM Udi Meiri <eh...@google.com> wrote:
>>
>>> To follow up on the Jenkins Job Cacher Plugin:
>>>
>>> Using a Jenkins plugin to save and reuse the Gradle cache for successive
>>> precommit jobs.
>>> The problem with this approach is that the precommit runs that a Jenkins
>>> server runs are unrelated.
>>> Say you have 2 PRs, A and B, and the precommit job for B reuses the
>>> cache left by the job for A.
>>> The diff between the two will cause tests affected both by A and B to be
>>> rerun (at least).
>>> If A modifies Python code, then the job for B must rerun ALL Python
>>> tests (since Gradle doesn't do dependency tracking for Python).
>>>
>>> Proposal:
>>> a. The cache plugin is still useful for successive Java precommit jobs,
>>> but not for Python. (Go, I have no idea)
>>> We could use it exclusively for Java precommits.
>>> b. To avoid running precommit jobs for code not touched by a PR, look at
>>> the paths of files changed.
>>> For example, a PR touching only files under sdks/python/... need only
>>> run Python precommit tests.
>>>
>>> On Tue, Jun 5, 2018 at 7:24 PM Udi Meiri <eh...@google.com> wrote:
>>>
>>>> I've been having a separate discussion on the proposal doc, which is
>>>> ready for another round of reviews.
>>>> Change summary:
>>>> - Changed fast requirement to be < 30 minutes and simplify the check as
>>>> an aggregate for each precommit job type.
>>>> - Updated slowness notification methods to include automated methods:
>>>> as a precommit check result type on GitHub, as a bug.
>>>> - Merged in the metrics design doc.
>>>> - Added detailed design section.
>>>> - Added list of deliverables.
>>>>
>>>> What I would like is consensus regarding:
>>>> - How fast we want precommit runs to be. I propose 30m.
>>>> - Deadline for fixing a slow test before it is temporarily removed from
>>>> precommit. I propose 24 hours.
>>>>
>>>>
>>>> Replying to the thread:
>>>>
>>>> 1. I like the idea of using the Jenkins Job Cacher Plugin to skip
>>>> unaffected tests (BEAM-4400).
>>>>
>>>> 2. Java Precommit tests include integration tests (example
>>>> <https://builds.apache.org/view/A-D/view/Beam/job/beam_PreCommit_Java_GradleBuild/lastCompletedBuild/testReport/org.apache.beam.examples/>
>>>> ).
>>>> We could split these out to get much faster results, i.e., a separate
>>>> precommit just for basic integration tests (which will still need to run in
>>>> <30m).
>>>> Perhaps lint checks for Python could be split out as well.
>>>>
>>>> I'll add these suggestions to the doc tomorrow.
>>>>
>>>> On Thu, May 24, 2018 at 9:25 AM Scott Wegner <sw...@google.com>
>>>> wrote:
>>>>
>>>>> So, it sounds like there's agreement that we should improve precommit
>>>>> times by only running necessary tests, and configuring Jenkins Job
>>>>> Caching + Gradle build cache is a path to get there. I've filed BEAM-4400
>>>>> [1] to follow-up on this.
>>>>>
>>>>> Getting back to Udi's original proposal [2]: I see value in defining a
>>>>> metric and target for overall pre-commit timing. The proposal for an
>>>>> initial "2 hour" target is helpful as a guardrail: we're already hitting
>>>>> it, but if we drift to a point where we're not, that should trigger some
>>>>> action to be taken to get back to a healthy state.
>>>>>
>>>>> I wouldn't mind separately setting a more aspiration goal of getting
>>>>> the pre-commits even faster (i.e. 15-30 mins), but I suspect that would
>>>>> require a concerted effort to evaluate and improve existing tests across
>>>>> the codebase. One idea would be to set up ensure the metric reporting can
>>>>> show the trend, and which tests are responsible for the most walltime, so
>>>>> that we know where to invest any efforts to improve tests.
>>>>>
>>>>>
>>>>> [1] https://issues.apache.org/jira/browse/BEAM-4400
>>>>> [2]
>>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>>
>>>>>
>>>>> On Wed, May 23, 2018 at 11:46 AM Kenneth Knowles <kl...@google.com>
>>>>> wrote:
>>>>>
>>>>>> With regard to the Job Cacher Plugin: I think it is an infra ticket
>>>>>> to install? And I guess we need it longer term when we move to
>>>>>> containerized builds anyhow? One thing I've experienced with the Travis-CI
>>>>>> cache is that the time spent uploading & downloading the remote cache - in
>>>>>> that case of all the pip installed dependencies - negated the benefits.
>>>>>> Probably for Beam it will have a greater benefit if we can skip most of the
>>>>>> build.
>>>>>>
>>>>>> Regarding integration tests in precommit: I think it is OK to run
>>>>>> maybe one Dataflow job in precommit, but it should be in parallel with the
>>>>>> unit tests and just a smoke test that takes 5 minutes, not a suite that
>>>>>> takes 35 minutes. So IMO that is low-hanging fruit. If this would make
>>>>>> postcommit unstable, then it also means precommit is unstable. Both are
>>>>>> troublesome.
>>>>>>
>>>>>> More short term, some possible hacks:
>>>>>>
>>>>>>  - Point gradle to cache outside the git workspace. We already did
>>>>>> this for .m2 and it helped a lot.
>>>>>>  - Intersect touched files with projects. Our nonstandard project
>>>>>> names might be a pain here. Not sure if fixing that is on the roadmap.
>>>>>>
>>>>>> Kenn
>>>>>>
>>>>>> On Wed, May 23, 2018 at 9:31 AM Ismaël Mejía <ie...@gmail.com>
>>>>>> wrote:
>>>>>>
>>>>>>> I second Robert idea of ‘inteligently’ running only the affected
>>>>>>> tests,
>>>>>>> probably
>>>>>>> there is no need to run Java for a go fix (and eventually if any
>>>>>>> issue it
>>>>>>> can be
>>>>>>> catched in postcommit), same for a dev who just fixed something in
>>>>>>> KafkaIO
>>>>>>> and has
>>>>>>> to wait for other IO tests to pass. I suppose that languages, IOs and
>>>>>>> extensions
>>>>>>> are ‘easy’ to isolate so maybe we can start with those.
>>>>>>>
>>>>>>> Earlier signals are also definitely great to have too, but not sure
>>>>>>> how we
>>>>>>> can
>>>>>>> have those with the current infra.
>>>>>>>
>>>>>>>  From a quicklook the biggest time is consumed by the examples module
>>>>>>> probably
>>>>>>> because they run in Dataflow with real IOs no?, that module alone
>>>>>>> takes ~35
>>>>>>> minutes, so maybe moving it to postcommit will gain us some quick
>>>>>>> improvement.
>>>>>>> On the other hand we should probably not dismiss the consequences of
>>>>>>> moving
>>>>>>> more
>>>>>>> stuff to postcommit given that our current postcommit is not the most
>>>>>>> stable, or
>>>>>>> the quickest, only the Dataflow suite takes 1h30!
>>>>>>>
>>>>>>>
>>>>>>> On Tue, May 22, 2018 at 12:01 AM Mikhail Gryzykhin <
>>>>>>> migryz@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> > What we can do here is estimate how much effort we want to put in
>>>>>>> and set
>>>>>>> remote target.
>>>>>>> > Such as:
>>>>>>> > Third quarter 2018 -- 1hr SLO
>>>>>>> > Forth quarter 2018 -- 30min SLO,
>>>>>>> > etc.
>>>>>>>
>>>>>>> > Combined with policy for newly added tests, this can give us some
>>>>>>> goal to
>>>>>>> aim for.
>>>>>>>
>>>>>>> > --Mikhail
>>>>>>>
>>>>>>> > Have feedback?
>>>>>>>
>>>>>>>
>>>>>>> > On Mon, May 21, 2018 at 2:06 PM Scott Wegner <sw...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >> Thanks for the proposal, I left comments in the doc. Overall I
>>>>>>> think
>>>>>>> it's a great idea.
>>>>>>>
>>>>>>> >> I've seen other projects with much faster pre-commits, and it
>>>>>>> requires
>>>>>>> strict guidelines on unit test design and keeping tests isolated
>>>>>>> in-memory
>>>>>>> as much as possible. That's not currently the case in Java; we have
>>>>>>> pre-commits which submit pipelines to Dataflow service.
>>>>>>>
>>>>>>> >> I don't know if it's feasible to get Java down to 15-20 mins in
>>>>>>> the
>>>>>>> short term, but a good starting point would be to document the
>>>>>>> requirements
>>>>>>> for a test to run as pre-commit, and start enforcing it for new
>>>>>>> tests.
>>>>>>>
>>>>>>>
>>>>>>> >> On Fri, May 18, 2018 at 3:25 PM Henning Rohde <he...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>> Good proposal. I think it should be considered in tandem with
>>>>>>> the "No
>>>>>>> commit on red post-commit" proposal and could be far more ambitious
>>>>>>> than 2
>>>>>>> hours. For example, something in the <15-20 mins range, say, would
>>>>>>> be much
>>>>>>> less of an inconvenience to the development effort. Go takes ~3
>>>>>>> mins, which
>>>>>>> means that it is practical to wait until a PR is green before asking
>>>>>>> anyone
>>>>>>> to look at it. If I need to wait for a Java or Python pre-commit, I
>>>>>>> task
>>>>>>> switch and come back later. If the post-commits are enforced to be
>>>>>>> green,
>>>>>>> we could possibly gain a much more productive flow at the cost of the
>>>>>>> occasional post-commit break, compared to now. Maybe IOs can be less
>>>>>>> extensively tested pre-commit, for example, or only if actually
>>>>>>> changed?
>>>>>>>
>>>>>>> >>> I also like Robert's suggestion of spitting up pre-commits into
>>>>>>> something more fine-grained to get a clear partial signal quicker.
>>>>>>> If we
>>>>>>> have an adequate number of Jenkins slots, it might also speed things
>>>>>>> up
>>>>>>> overall.
>>>>>>>
>>>>>>> >>> Thanks,
>>>>>>> >>>    Henning
>>>>>>>
>>>>>>> >>> On Fri, May 18, 2018 at 12:30 PM Scott Wegner <
>>>>>>> swegner@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>> re: intelligently skipping tests for code that doesn't change
>>>>>>> (i.e.
>>>>>>> Java tests on Python PR): this should be possible. We already have
>>>>>>> build-caching enabled in Gradle, but I believe it is local to the git
>>>>>>> workspace and doesn't persist between Jenkins runs.
>>>>>>>
>>>>>>> >>>> With a quick search, I see there is a Jenkins Build Cacher
>>>>>>> Plugin [1]
>>>>>>> that hooks into Gradle build cache and does exactly what we need.
>>>>>>> Does
>>>>>>> anybody know whether we could get this enabled on our Jenkins?
>>>>>>>
>>>>>>> >>>> [1] https://wiki.jenkins.io/display/JENKINS/Job+Cacher+Plugin
>>>>>>>
>>>>>>> >>>> On Fri, May 18, 2018 at 12:08 PM Robert Bradshaw <
>>>>>>> robertwb@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>>> [somehow  my email got garbled...]
>>>>>>>
>>>>>>> >>>>> Now that we're using gradle, perhaps we could be more
>>>>>>> intelligent
>>>>>>> about only running the affected tests? E.g. when you touch Python
>>>>>>> (or Go)
>>>>>>> you shouldn't need to run the Java precommit at all, which would
>>>>>>> reduce the
>>>>>>> latency for those PRs and also the time spent in queue. Presumably
>>>>>>> this
>>>>>>> could even be applied per-module for the Java tests. (Maybe a large,
>>>>>>> shared
>>>>>>> build cache could help here as well...)
>>>>>>>
>>>>>>> >>>>> I also wouldn't be opposed to a quicker immediate signal, plus
>>>>>>> more
>>>>>>> extensive tests before actually merging. It's also nice to not have
>>>>>>> to wait
>>>>>>> an hour to see that you have a lint error; quick stuff like that
>>>>>>> could be
>>>>>>> signaled quickly before a contributor looses context.
>>>>>>>
>>>>>>> >>>>> - Robert
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> >>>>> On Fri, May 18, 2018 at 5:55 AM Kenneth Knowles <
>>>>>>> klk@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>>>> I like the idea. I think it is a good time for the project to
>>>>>>> start
>>>>>>> tracking this and keeping it usable.
>>>>>>>
>>>>>>> >>>>>> Certainly 2 hours is more than enough, is that not so? The
>>>>>>> Java
>>>>>>> precommit seems to take <=40 minutes while Python takes ~20 and Go
>>>>>>> is so
>>>>>>> fast it doesn't matter. Do we have enough stragglers that we don't
>>>>>>> make it
>>>>>>> in the 95th percentile? Is the time spent in the Jenkins queue?
>>>>>>>
>>>>>>> >>>>>> For our current coverage, I'd be willing to go for:
>>>>>>>
>>>>>>> >>>>>>    - 1 hr hard cap (someone better at stats could choose %ile)
>>>>>>> >>>>>>    - roll back or remove test from precommit if fix looks
>>>>>>> like more
>>>>>>> than 1 week (roll back if it is perf degradation, remove test from
>>>>>>> precommit if it is additional coverage that just doesn't fit in the
>>>>>>> time)
>>>>>>>
>>>>>>> >>>>>> There's a longer-term issue that doing a full build each time
>>>>>>> is
>>>>>>> expected to linearly scale up with the size of our repo (it is the
>>>>>>> monorepo
>>>>>>> problem but for a minirepo) so there is no cap that is feasible
>>>>>>> until we
>>>>>>> have effective cross-build caching. And my long-term goal would be
>>>>>>> <30
>>>>>>> minutes. At the latency of opening a pull request and then checking
>>>>>>> your
>>>>>>> email that's not burdensome, but an hour is.
>>>>>>>
>>>>>>> >>>>>> Kenn
>>>>>>>
>>>>>>> >>>>>> On Thu, May 17, 2018 at 6:54 PM Udi Meiri <eh...@google.com>
>>>>>>> wrote:
>>>>>>>
>>>>>>> >>>>>>> HI,
>>>>>>> >>>>>>> I have a proposal to improve contributor experience by
>>>>>>> keeping
>>>>>>> precommit times low.
>>>>>>>
>>>>>>> >>>>>>> I'm looking to get community consensus and approval about:
>>>>>>> >>>>>>> 1. How long should precommits take. 2 hours @95th percentile
>>>>>>> over
>>>>>>> the past 4 weeks is the current proposal.
>>>>>>> >>>>>>> 2. The process for dealing with slowness. Do we: fix, roll
>>>>>>> back,
>>>>>>> remove a test from precommit?
>>>>>>> >>>>>>> Rolling back if a fix is estimated to take longer than 2
>>>>>>> weeks is
>>>>>>> the current proposal.
>>>>>>>
>>>>>>>
>>>>>>>
>>>>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>>>>
>>>>>>

Re: Proposal: keeping precommit times fast

Posted by Udi Meiri <eh...@google.com>.
Would I need a vote on installing this plugin, or can I just open a ticket
to infra?

On Wed, Jun 6, 2018, 16:18 Robert Bradshaw <ro...@google.com> wrote:

> Even if it's not perfect, seems like it'd surely be a net win (and
> probably a large one). Also, the build cache should look back at more than
> just the single previous build, so if any previous jobs (up to the cache
> size limit) built/tested artifacts unchanged by the current PR, the results
> would live in the cache.
>
> I would look at (a) and (b) only if this isn't already good enough.
>
> On Wed, Jun 6, 2018 at 3:50 PM Udi Meiri <eh...@google.com> wrote:
>
>> To follow up on the Jenkins Job Cacher Plugin:
>>
>> Using a Jenkins plugin to save and reuse the Gradle cache for successive
>> precommit jobs.
>> The problem with this approach is that the precommit runs that a Jenkins
>> server runs are unrelated.
>> Say you have 2 PRs, A and B, and the precommit job for B reuses the cache
>> left by the job for A.
>> The diff between the two will cause tests affected both by A and B to be
>> rerun (at least).
>> If A modifies Python code, then the job for B must rerun ALL Python tests
>> (since Gradle doesn't do dependency tracking for Python).
>>
>> Proposal:
>> a. The cache plugin is still useful for successive Java precommit jobs,
>> but not for Python. (Go, I have no idea)
>> We could use it exclusively for Java precommits.
>> b. To avoid running precommit jobs for code not touched by a PR, look at
>> the paths of files changed.
>> For example, a PR touching only files under sdks/python/... need only run
>> Python precommit tests.
>>
>> On Tue, Jun 5, 2018 at 7:24 PM Udi Meiri <eh...@google.com> wrote:
>>
>>> I've been having a separate discussion on the proposal doc, which is
>>> ready for another round of reviews.
>>> Change summary:
>>> - Changed fast requirement to be < 30 minutes and simplify the check as
>>> an aggregate for each precommit job type.
>>> - Updated slowness notification methods to include automated methods: as
>>> a precommit check result type on GitHub, as a bug.
>>> - Merged in the metrics design doc.
>>> - Added detailed design section.
>>> - Added list of deliverables.
>>>
>>> What I would like is consensus regarding:
>>> - How fast we want precommit runs to be. I propose 30m.
>>> - Deadline for fixing a slow test before it is temporarily removed from
>>> precommit. I propose 24 hours.
>>>
>>>
>>> Replying to the thread:
>>>
>>> 1. I like the idea of using the Jenkins Job Cacher Plugin to skip
>>> unaffected tests (BEAM-4400).
>>>
>>> 2. Java Precommit tests include integration tests (example
>>> <https://builds.apache.org/view/A-D/view/Beam/job/beam_PreCommit_Java_GradleBuild/lastCompletedBuild/testReport/org.apache.beam.examples/>
>>> ).
>>> We could split these out to get much faster results, i.e., a separate
>>> precommit just for basic integration tests (which will still need to run in
>>> <30m).
>>> Perhaps lint checks for Python could be split out as well.
>>>
>>> I'll add these suggestions to the doc tomorrow.
>>>
>>> On Thu, May 24, 2018 at 9:25 AM Scott Wegner <sw...@google.com> wrote:
>>>
>>>> So, it sounds like there's agreement that we should improve precommit
>>>> times by only running necessary tests, and configuring Jenkins Job
>>>> Caching + Gradle build cache is a path to get there. I've filed BEAM-4400
>>>> [1] to follow-up on this.
>>>>
>>>> Getting back to Udi's original proposal [2]: I see value in defining a
>>>> metric and target for overall pre-commit timing. The proposal for an
>>>> initial "2 hour" target is helpful as a guardrail: we're already hitting
>>>> it, but if we drift to a point where we're not, that should trigger some
>>>> action to be taken to get back to a healthy state.
>>>>
>>>> I wouldn't mind separately setting a more aspiration goal of getting
>>>> the pre-commits even faster (i.e. 15-30 mins), but I suspect that would
>>>> require a concerted effort to evaluate and improve existing tests across
>>>> the codebase. One idea would be to set up ensure the metric reporting can
>>>> show the trend, and which tests are responsible for the most walltime, so
>>>> that we know where to invest any efforts to improve tests.
>>>>
>>>>
>>>> [1] https://issues.apache.org/jira/browse/BEAM-4400
>>>> [2]
>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>
>>>>
>>>> On Wed, May 23, 2018 at 11:46 AM Kenneth Knowles <kl...@google.com>
>>>> wrote:
>>>>
>>>>> With regard to the Job Cacher Plugin: I think it is an infra ticket to
>>>>> install? And I guess we need it longer term when we move to containerized
>>>>> builds anyhow? One thing I've experienced with the Travis-CI cache is that
>>>>> the time spent uploading & downloading the remote cache - in that case of
>>>>> all the pip installed dependencies - negated the benefits. Probably for
>>>>> Beam it will have a greater benefit if we can skip most of the build.
>>>>>
>>>>> Regarding integration tests in precommit: I think it is OK to run
>>>>> maybe one Dataflow job in precommit, but it should be in parallel with the
>>>>> unit tests and just a smoke test that takes 5 minutes, not a suite that
>>>>> takes 35 minutes. So IMO that is low-hanging fruit. If this would make
>>>>> postcommit unstable, then it also means precommit is unstable. Both are
>>>>> troublesome.
>>>>>
>>>>> More short term, some possible hacks:
>>>>>
>>>>>  - Point gradle to cache outside the git workspace. We already did
>>>>> this for .m2 and it helped a lot.
>>>>>  - Intersect touched files with projects. Our nonstandard project
>>>>> names might be a pain here. Not sure if fixing that is on the roadmap.
>>>>>
>>>>> Kenn
>>>>>
>>>>> On Wed, May 23, 2018 at 9:31 AM Ismaël Mejía <ie...@gmail.com>
>>>>> wrote:
>>>>>
>>>>>> I second Robert idea of ‘inteligently’ running only the affected
>>>>>> tests,
>>>>>> probably
>>>>>> there is no need to run Java for a go fix (and eventually if any
>>>>>> issue it
>>>>>> can be
>>>>>> catched in postcommit), same for a dev who just fixed something in
>>>>>> KafkaIO
>>>>>> and has
>>>>>> to wait for other IO tests to pass. I suppose that languages, IOs and
>>>>>> extensions
>>>>>> are ‘easy’ to isolate so maybe we can start with those.
>>>>>>
>>>>>> Earlier signals are also definitely great to have too, but not sure
>>>>>> how we
>>>>>> can
>>>>>> have those with the current infra.
>>>>>>
>>>>>>  From a quicklook the biggest time is consumed by the examples module
>>>>>> probably
>>>>>> because they run in Dataflow with real IOs no?, that module alone
>>>>>> takes ~35
>>>>>> minutes, so maybe moving it to postcommit will gain us some quick
>>>>>> improvement.
>>>>>> On the other hand we should probably not dismiss the consequences of
>>>>>> moving
>>>>>> more
>>>>>> stuff to postcommit given that our current postcommit is not the most
>>>>>> stable, or
>>>>>> the quickest, only the Dataflow suite takes 1h30!
>>>>>>
>>>>>>
>>>>>> On Tue, May 22, 2018 at 12:01 AM Mikhail Gryzykhin <migryz@google.com
>>>>>> >
>>>>>> wrote:
>>>>>>
>>>>>> > What we can do here is estimate how much effort we want to put in
>>>>>> and set
>>>>>> remote target.
>>>>>> > Such as:
>>>>>> > Third quarter 2018 -- 1hr SLO
>>>>>> > Forth quarter 2018 -- 30min SLO,
>>>>>> > etc.
>>>>>>
>>>>>> > Combined with policy for newly added tests, this can give us some
>>>>>> goal to
>>>>>> aim for.
>>>>>>
>>>>>> > --Mikhail
>>>>>>
>>>>>> > Have feedback?
>>>>>>
>>>>>>
>>>>>> > On Mon, May 21, 2018 at 2:06 PM Scott Wegner <sw...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>> >> Thanks for the proposal, I left comments in the doc. Overall I
>>>>>> think
>>>>>> it's a great idea.
>>>>>>
>>>>>> >> I've seen other projects with much faster pre-commits, and it
>>>>>> requires
>>>>>> strict guidelines on unit test design and keeping tests isolated
>>>>>> in-memory
>>>>>> as much as possible. That's not currently the case in Java; we have
>>>>>> pre-commits which submit pipelines to Dataflow service.
>>>>>>
>>>>>> >> I don't know if it's feasible to get Java down to 15-20 mins in the
>>>>>> short term, but a good starting point would be to document the
>>>>>> requirements
>>>>>> for a test to run as pre-commit, and start enforcing it for new tests.
>>>>>>
>>>>>>
>>>>>> >> On Fri, May 18, 2018 at 3:25 PM Henning Rohde <he...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>> >>> Good proposal. I think it should be considered in tandem with the
>>>>>> "No
>>>>>> commit on red post-commit" proposal and could be far more ambitious
>>>>>> than 2
>>>>>> hours. For example, something in the <15-20 mins range, say, would be
>>>>>> much
>>>>>> less of an inconvenience to the development effort. Go takes ~3 mins,
>>>>>> which
>>>>>> means that it is practical to wait until a PR is green before asking
>>>>>> anyone
>>>>>> to look at it. If I need to wait for a Java or Python pre-commit, I
>>>>>> task
>>>>>> switch and come back later. If the post-commits are enforced to be
>>>>>> green,
>>>>>> we could possibly gain a much more productive flow at the cost of the
>>>>>> occasional post-commit break, compared to now. Maybe IOs can be less
>>>>>> extensively tested pre-commit, for example, or only if actually
>>>>>> changed?
>>>>>>
>>>>>> >>> I also like Robert's suggestion of spitting up pre-commits into
>>>>>> something more fine-grained to get a clear partial signal quicker. If
>>>>>> we
>>>>>> have an adequate number of Jenkins slots, it might also speed things
>>>>>> up
>>>>>> overall.
>>>>>>
>>>>>> >>> Thanks,
>>>>>> >>>    Henning
>>>>>>
>>>>>> >>> On Fri, May 18, 2018 at 12:30 PM Scott Wegner <swegner@google.com
>>>>>> >
>>>>>> wrote:
>>>>>>
>>>>>> >>>> re: intelligently skipping tests for code that doesn't change
>>>>>> (i.e.
>>>>>> Java tests on Python PR): this should be possible. We already have
>>>>>> build-caching enabled in Gradle, but I believe it is local to the git
>>>>>> workspace and doesn't persist between Jenkins runs.
>>>>>>
>>>>>> >>>> With a quick search, I see there is a Jenkins Build Cacher
>>>>>> Plugin [1]
>>>>>> that hooks into Gradle build cache and does exactly what we need. Does
>>>>>> anybody know whether we could get this enabled on our Jenkins?
>>>>>>
>>>>>> >>>> [1] https://wiki.jenkins.io/display/JENKINS/Job+Cacher+Plugin
>>>>>>
>>>>>> >>>> On Fri, May 18, 2018 at 12:08 PM Robert Bradshaw <
>>>>>> robertwb@google.com>
>>>>>> wrote:
>>>>>>
>>>>>> >>>>> [somehow  my email got garbled...]
>>>>>>
>>>>>> >>>>> Now that we're using gradle, perhaps we could be more
>>>>>> intelligent
>>>>>> about only running the affected tests? E.g. when you touch Python (or
>>>>>> Go)
>>>>>> you shouldn't need to run the Java precommit at all, which would
>>>>>> reduce the
>>>>>> latency for those PRs and also the time spent in queue. Presumably
>>>>>> this
>>>>>> could even be applied per-module for the Java tests. (Maybe a large,
>>>>>> shared
>>>>>> build cache could help here as well...)
>>>>>>
>>>>>> >>>>> I also wouldn't be opposed to a quicker immediate signal, plus
>>>>>> more
>>>>>> extensive tests before actually merging. It's also nice to not have
>>>>>> to wait
>>>>>> an hour to see that you have a lint error; quick stuff like that
>>>>>> could be
>>>>>> signaled quickly before a contributor looses context.
>>>>>>
>>>>>> >>>>> - Robert
>>>>>>
>>>>>>
>>>>>>
>>>>>> >>>>> On Fri, May 18, 2018 at 5:55 AM Kenneth Knowles <klk@google.com
>>>>>> >
>>>>>> wrote:
>>>>>>
>>>>>> >>>>>> I like the idea. I think it is a good time for the project to
>>>>>> start
>>>>>> tracking this and keeping it usable.
>>>>>>
>>>>>> >>>>>> Certainly 2 hours is more than enough, is that not so? The Java
>>>>>> precommit seems to take <=40 minutes while Python takes ~20 and Go is
>>>>>> so
>>>>>> fast it doesn't matter. Do we have enough stragglers that we don't
>>>>>> make it
>>>>>> in the 95th percentile? Is the time spent in the Jenkins queue?
>>>>>>
>>>>>> >>>>>> For our current coverage, I'd be willing to go for:
>>>>>>
>>>>>> >>>>>>    - 1 hr hard cap (someone better at stats could choose %ile)
>>>>>> >>>>>>    - roll back or remove test from precommit if fix looks like
>>>>>> more
>>>>>> than 1 week (roll back if it is perf degradation, remove test from
>>>>>> precommit if it is additional coverage that just doesn't fit in the
>>>>>> time)
>>>>>>
>>>>>> >>>>>> There's a longer-term issue that doing a full build each time
>>>>>> is
>>>>>> expected to linearly scale up with the size of our repo (it is the
>>>>>> monorepo
>>>>>> problem but for a minirepo) so there is no cap that is feasible until
>>>>>> we
>>>>>> have effective cross-build caching. And my long-term goal would be <30
>>>>>> minutes. At the latency of opening a pull request and then checking
>>>>>> your
>>>>>> email that's not burdensome, but an hour is.
>>>>>>
>>>>>> >>>>>> Kenn
>>>>>>
>>>>>> >>>>>> On Thu, May 17, 2018 at 6:54 PM Udi Meiri <eh...@google.com>
>>>>>> wrote:
>>>>>>
>>>>>> >>>>>>> HI,
>>>>>> >>>>>>> I have a proposal to improve contributor experience by keeping
>>>>>> precommit times low.
>>>>>>
>>>>>> >>>>>>> I'm looking to get community consensus and approval about:
>>>>>> >>>>>>> 1. How long should precommits take. 2 hours @95th percentile
>>>>>> over
>>>>>> the past 4 weeks is the current proposal.
>>>>>> >>>>>>> 2. The process for dealing with slowness. Do we: fix, roll
>>>>>> back,
>>>>>> remove a test from precommit?
>>>>>> >>>>>>> Rolling back if a fix is estimated to take longer than 2
>>>>>> weeks is
>>>>>> the current proposal.
>>>>>>
>>>>>>
>>>>>>
>>>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>>>
>>>>>

Re: Proposal: keeping precommit times fast

Posted by Robert Bradshaw <ro...@google.com>.
Even if it's not perfect, seems like it'd surely be a net win (and probably
a large one). Also, the build cache should look back at more than just the
single previous build, so if any previous jobs (up to the cache size limit)
built/tested artifacts unchanged by the current PR, the results would live
in the cache.

I would look at (a) and (b) only if this isn't already good enough.

On Wed, Jun 6, 2018 at 3:50 PM Udi Meiri <eh...@google.com> wrote:

> To follow up on the Jenkins Job Cacher Plugin:
>
> Using a Jenkins plugin to save and reuse the Gradle cache for successive
> precommit jobs.
> The problem with this approach is that the precommit runs that a Jenkins
> server runs are unrelated.
> Say you have 2 PRs, A and B, and the precommit job for B reuses the cache
> left by the job for A.
> The diff between the two will cause tests affected both by A and B to be
> rerun (at least).
> If A modifies Python code, then the job for B must rerun ALL Python tests
> (since Gradle doesn't do dependency tracking for Python).
>
> Proposal:
> a. The cache plugin is still useful for successive Java precommit jobs,
> but not for Python. (Go, I have no idea)
> We could use it exclusively for Java precommits.
> b. To avoid running precommit jobs for code not touched by a PR, look at
> the paths of files changed.
> For example, a PR touching only files under sdks/python/... need only run
> Python precommit tests.
>
> On Tue, Jun 5, 2018 at 7:24 PM Udi Meiri <eh...@google.com> wrote:
>
>> I've been having a separate discussion on the proposal doc, which is
>> ready for another round of reviews.
>> Change summary:
>> - Changed fast requirement to be < 30 minutes and simplify the check as
>> an aggregate for each precommit job type.
>> - Updated slowness notification methods to include automated methods: as
>> a precommit check result type on GitHub, as a bug.
>> - Merged in the metrics design doc.
>> - Added detailed design section.
>> - Added list of deliverables.
>>
>> What I would like is consensus regarding:
>> - How fast we want precommit runs to be. I propose 30m.
>> - Deadline for fixing a slow test before it is temporarily removed from
>> precommit. I propose 24 hours.
>>
>>
>> Replying to the thread:
>>
>> 1. I like the idea of using the Jenkins Job Cacher Plugin to skip
>> unaffected tests (BEAM-4400).
>>
>> 2. Java Precommit tests include integration tests (example
>> <https://builds.apache.org/view/A-D/view/Beam/job/beam_PreCommit_Java_GradleBuild/lastCompletedBuild/testReport/org.apache.beam.examples/>
>> ).
>> We could split these out to get much faster results, i.e., a separate
>> precommit just for basic integration tests (which will still need to run in
>> <30m).
>> Perhaps lint checks for Python could be split out as well.
>>
>> I'll add these suggestions to the doc tomorrow.
>>
>> On Thu, May 24, 2018 at 9:25 AM Scott Wegner <sw...@google.com> wrote:
>>
>>> So, it sounds like there's agreement that we should improve precommit
>>> times by only running necessary tests, and configuring Jenkins Job
>>> Caching + Gradle build cache is a path to get there. I've filed BEAM-4400
>>> [1] to follow-up on this.
>>>
>>> Getting back to Udi's original proposal [2]: I see value in defining a
>>> metric and target for overall pre-commit timing. The proposal for an
>>> initial "2 hour" target is helpful as a guardrail: we're already hitting
>>> it, but if we drift to a point where we're not, that should trigger some
>>> action to be taken to get back to a healthy state.
>>>
>>> I wouldn't mind separately setting a more aspiration goal of getting the
>>> pre-commits even faster (i.e. 15-30 mins), but I suspect that would require
>>> a concerted effort to evaluate and improve existing tests across the
>>> codebase. One idea would be to set up ensure the metric reporting can show
>>> the trend, and which tests are responsible for the most walltime, so that
>>> we know where to invest any efforts to improve tests.
>>>
>>>
>>> [1] https://issues.apache.org/jira/browse/BEAM-4400
>>> [2]
>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>
>>>
>>> On Wed, May 23, 2018 at 11:46 AM Kenneth Knowles <kl...@google.com> wrote:
>>>
>>>> With regard to the Job Cacher Plugin: I think it is an infra ticket to
>>>> install? And I guess we need it longer term when we move to containerized
>>>> builds anyhow? One thing I've experienced with the Travis-CI cache is that
>>>> the time spent uploading & downloading the remote cache - in that case of
>>>> all the pip installed dependencies - negated the benefits. Probably for
>>>> Beam it will have a greater benefit if we can skip most of the build.
>>>>
>>>> Regarding integration tests in precommit: I think it is OK to run maybe
>>>> one Dataflow job in precommit, but it should be in parallel with the unit
>>>> tests and just a smoke test that takes 5 minutes, not a suite that takes 35
>>>> minutes. So IMO that is low-hanging fruit. If this would make postcommit
>>>> unstable, then it also means precommit is unstable. Both are troublesome.
>>>>
>>>> More short term, some possible hacks:
>>>>
>>>>  - Point gradle to cache outside the git workspace. We already did this
>>>> for .m2 and it helped a lot.
>>>>  - Intersect touched files with projects. Our nonstandard project names
>>>> might be a pain here. Not sure if fixing that is on the roadmap.
>>>>
>>>> Kenn
>>>>
>>>> On Wed, May 23, 2018 at 9:31 AM Ismaël Mejía <ie...@gmail.com> wrote:
>>>>
>>>>> I second Robert idea of ‘inteligently’ running only the affected tests,
>>>>> probably
>>>>> there is no need to run Java for a go fix (and eventually if any issue
>>>>> it
>>>>> can be
>>>>> catched in postcommit), same for a dev who just fixed something in
>>>>> KafkaIO
>>>>> and has
>>>>> to wait for other IO tests to pass. I suppose that languages, IOs and
>>>>> extensions
>>>>> are ‘easy’ to isolate so maybe we can start with those.
>>>>>
>>>>> Earlier signals are also definitely great to have too, but not sure
>>>>> how we
>>>>> can
>>>>> have those with the current infra.
>>>>>
>>>>>  From a quicklook the biggest time is consumed by the examples module
>>>>> probably
>>>>> because they run in Dataflow with real IOs no?, that module alone
>>>>> takes ~35
>>>>> minutes, so maybe moving it to postcommit will gain us some quick
>>>>> improvement.
>>>>> On the other hand we should probably not dismiss the consequences of
>>>>> moving
>>>>> more
>>>>> stuff to postcommit given that our current postcommit is not the most
>>>>> stable, or
>>>>> the quickest, only the Dataflow suite takes 1h30!
>>>>>
>>>>>
>>>>> On Tue, May 22, 2018 at 12:01 AM Mikhail Gryzykhin <mi...@google.com>
>>>>> wrote:
>>>>>
>>>>> > What we can do here is estimate how much effort we want to put in
>>>>> and set
>>>>> remote target.
>>>>> > Such as:
>>>>> > Third quarter 2018 -- 1hr SLO
>>>>> > Forth quarter 2018 -- 30min SLO,
>>>>> > etc.
>>>>>
>>>>> > Combined with policy for newly added tests, this can give us some
>>>>> goal to
>>>>> aim for.
>>>>>
>>>>> > --Mikhail
>>>>>
>>>>> > Have feedback?
>>>>>
>>>>>
>>>>> > On Mon, May 21, 2018 at 2:06 PM Scott Wegner <sw...@google.com>
>>>>> wrote:
>>>>>
>>>>> >> Thanks for the proposal, I left comments in the doc. Overall I think
>>>>> it's a great idea.
>>>>>
>>>>> >> I've seen other projects with much faster pre-commits, and it
>>>>> requires
>>>>> strict guidelines on unit test design and keeping tests isolated
>>>>> in-memory
>>>>> as much as possible. That's not currently the case in Java; we have
>>>>> pre-commits which submit pipelines to Dataflow service.
>>>>>
>>>>> >> I don't know if it's feasible to get Java down to 15-20 mins in the
>>>>> short term, but a good starting point would be to document the
>>>>> requirements
>>>>> for a test to run as pre-commit, and start enforcing it for new tests.
>>>>>
>>>>>
>>>>> >> On Fri, May 18, 2018 at 3:25 PM Henning Rohde <he...@google.com>
>>>>> wrote:
>>>>>
>>>>> >>> Good proposal. I think it should be considered in tandem with the
>>>>> "No
>>>>> commit on red post-commit" proposal and could be far more ambitious
>>>>> than 2
>>>>> hours. For example, something in the <15-20 mins range, say, would be
>>>>> much
>>>>> less of an inconvenience to the development effort. Go takes ~3 mins,
>>>>> which
>>>>> means that it is practical to wait until a PR is green before asking
>>>>> anyone
>>>>> to look at it. If I need to wait for a Java or Python pre-commit, I
>>>>> task
>>>>> switch and come back later. If the post-commits are enforced to be
>>>>> green,
>>>>> we could possibly gain a much more productive flow at the cost of the
>>>>> occasional post-commit break, compared to now. Maybe IOs can be less
>>>>> extensively tested pre-commit, for example, or only if actually
>>>>> changed?
>>>>>
>>>>> >>> I also like Robert's suggestion of spitting up pre-commits into
>>>>> something more fine-grained to get a clear partial signal quicker. If
>>>>> we
>>>>> have an adequate number of Jenkins slots, it might also speed things up
>>>>> overall.
>>>>>
>>>>> >>> Thanks,
>>>>> >>>    Henning
>>>>>
>>>>> >>> On Fri, May 18, 2018 at 12:30 PM Scott Wegner <sw...@google.com>
>>>>> wrote:
>>>>>
>>>>> >>>> re: intelligently skipping tests for code that doesn't change
>>>>> (i.e.
>>>>> Java tests on Python PR): this should be possible. We already have
>>>>> build-caching enabled in Gradle, but I believe it is local to the git
>>>>> workspace and doesn't persist between Jenkins runs.
>>>>>
>>>>> >>>> With a quick search, I see there is a Jenkins Build Cacher Plugin
>>>>> [1]
>>>>> that hooks into Gradle build cache and does exactly what we need. Does
>>>>> anybody know whether we could get this enabled on our Jenkins?
>>>>>
>>>>> >>>> [1] https://wiki.jenkins.io/display/JENKINS/Job+Cacher+Plugin
>>>>>
>>>>> >>>> On Fri, May 18, 2018 at 12:08 PM Robert Bradshaw <
>>>>> robertwb@google.com>
>>>>> wrote:
>>>>>
>>>>> >>>>> [somehow  my email got garbled...]
>>>>>
>>>>> >>>>> Now that we're using gradle, perhaps we could be more intelligent
>>>>> about only running the affected tests? E.g. when you touch Python (or
>>>>> Go)
>>>>> you shouldn't need to run the Java precommit at all, which would
>>>>> reduce the
>>>>> latency for those PRs and also the time spent in queue. Presumably this
>>>>> could even be applied per-module for the Java tests. (Maybe a large,
>>>>> shared
>>>>> build cache could help here as well...)
>>>>>
>>>>> >>>>> I also wouldn't be opposed to a quicker immediate signal, plus
>>>>> more
>>>>> extensive tests before actually merging. It's also nice to not have to
>>>>> wait
>>>>> an hour to see that you have a lint error; quick stuff like that could
>>>>> be
>>>>> signaled quickly before a contributor looses context.
>>>>>
>>>>> >>>>> - Robert
>>>>>
>>>>>
>>>>>
>>>>> >>>>> On Fri, May 18, 2018 at 5:55 AM Kenneth Knowles <kl...@google.com>
>>>>> wrote:
>>>>>
>>>>> >>>>>> I like the idea. I think it is a good time for the project to
>>>>> start
>>>>> tracking this and keeping it usable.
>>>>>
>>>>> >>>>>> Certainly 2 hours is more than enough, is that not so? The Java
>>>>> precommit seems to take <=40 minutes while Python takes ~20 and Go is
>>>>> so
>>>>> fast it doesn't matter. Do we have enough stragglers that we don't
>>>>> make it
>>>>> in the 95th percentile? Is the time spent in the Jenkins queue?
>>>>>
>>>>> >>>>>> For our current coverage, I'd be willing to go for:
>>>>>
>>>>> >>>>>>    - 1 hr hard cap (someone better at stats could choose %ile)
>>>>> >>>>>>    - roll back or remove test from precommit if fix looks like
>>>>> more
>>>>> than 1 week (roll back if it is perf degradation, remove test from
>>>>> precommit if it is additional coverage that just doesn't fit in the
>>>>> time)
>>>>>
>>>>> >>>>>> There's a longer-term issue that doing a full build each time is
>>>>> expected to linearly scale up with the size of our repo (it is the
>>>>> monorepo
>>>>> problem but for a minirepo) so there is no cap that is feasible until
>>>>> we
>>>>> have effective cross-build caching. And my long-term goal would be <30
>>>>> minutes. At the latency of opening a pull request and then checking
>>>>> your
>>>>> email that's not burdensome, but an hour is.
>>>>>
>>>>> >>>>>> Kenn
>>>>>
>>>>> >>>>>> On Thu, May 17, 2018 at 6:54 PM Udi Meiri <eh...@google.com>
>>>>> wrote:
>>>>>
>>>>> >>>>>>> HI,
>>>>> >>>>>>> I have a proposal to improve contributor experience by keeping
>>>>> precommit times low.
>>>>>
>>>>> >>>>>>> I'm looking to get community consensus and approval about:
>>>>> >>>>>>> 1. How long should precommits take. 2 hours @95th percentile
>>>>> over
>>>>> the past 4 weeks is the current proposal.
>>>>> >>>>>>> 2. The process for dealing with slowness. Do we: fix, roll
>>>>> back,
>>>>> remove a test from precommit?
>>>>> >>>>>>> Rolling back if a fix is estimated to take longer than 2 weeks
>>>>> is
>>>>> the current proposal.
>>>>>
>>>>>
>>>>>
>>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>>
>>>>

Re: Proposal: keeping precommit times fast

Posted by Udi Meiri <eh...@google.com>.
To follow up on the Jenkins Job Cacher Plugin:

Using a Jenkins plugin to save and reuse the Gradle cache for successive
precommit jobs.
The problem with this approach is that the precommit runs that a Jenkins
server runs are unrelated.
Say you have 2 PRs, A and B, and the precommit job for B reuses the cache
left by the job for A.
The diff between the two will cause tests affected both by A and B to be
rerun (at least).
If A modifies Python code, then the job for B must rerun ALL Python tests
(since Gradle doesn't do dependency tracking for Python).

Proposal:
a. The cache plugin is still useful for successive Java precommit jobs, but
not for Python. (Go, I have no idea)
We could use it exclusively for Java precommits.
b. To avoid running precommit jobs for code not touched by a PR, look at
the paths of files changed.
For example, a PR touching only files under sdks/python/... need only run
Python precommit tests.

On Tue, Jun 5, 2018 at 7:24 PM Udi Meiri <eh...@google.com> wrote:

> I've been having a separate discussion on the proposal doc, which is ready
> for another round of reviews.
> Change summary:
> - Changed fast requirement to be < 30 minutes and simplify the check as an
> aggregate for each precommit job type.
> - Updated slowness notification methods to include automated methods: as a
> precommit check result type on GitHub, as a bug.
> - Merged in the metrics design doc.
> - Added detailed design section.
> - Added list of deliverables.
>
> What I would like is consensus regarding:
> - How fast we want precommit runs to be. I propose 30m.
> - Deadline for fixing a slow test before it is temporarily removed from
> precommit. I propose 24 hours.
>
>
> Replying to the thread:
>
> 1. I like the idea of using the Jenkins Job Cacher Plugin to skip
> unaffected tests (BEAM-4400).
>
> 2. Java Precommit tests include integration tests (example
> <https://builds.apache.org/view/A-D/view/Beam/job/beam_PreCommit_Java_GradleBuild/lastCompletedBuild/testReport/org.apache.beam.examples/>
> ).
> We could split these out to get much faster results, i.e., a separate
> precommit just for basic integration tests (which will still need to run in
> <30m).
> Perhaps lint checks for Python could be split out as well.
>
> I'll add these suggestions to the doc tomorrow.
>
> On Thu, May 24, 2018 at 9:25 AM Scott Wegner <sw...@google.com> wrote:
>
>> So, it sounds like there's agreement that we should improve precommit
>> times by only running necessary tests, and configuring Jenkins Job
>> Caching + Gradle build cache is a path to get there. I've filed BEAM-4400
>> [1] to follow-up on this.
>>
>> Getting back to Udi's original proposal [2]: I see value in defining a
>> metric and target for overall pre-commit timing. The proposal for an
>> initial "2 hour" target is helpful as a guardrail: we're already hitting
>> it, but if we drift to a point where we're not, that should trigger some
>> action to be taken to get back to a healthy state.
>>
>> I wouldn't mind separately setting a more aspiration goal of getting the
>> pre-commits even faster (i.e. 15-30 mins), but I suspect that would require
>> a concerted effort to evaluate and improve existing tests across the
>> codebase. One idea would be to set up ensure the metric reporting can show
>> the trend, and which tests are responsible for the most walltime, so that
>> we know where to invest any efforts to improve tests.
>>
>>
>> [1] https://issues.apache.org/jira/browse/BEAM-4400
>> [2]
>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>
>>
>> On Wed, May 23, 2018 at 11:46 AM Kenneth Knowles <kl...@google.com> wrote:
>>
>>> With regard to the Job Cacher Plugin: I think it is an infra ticket to
>>> install? And I guess we need it longer term when we move to containerized
>>> builds anyhow? One thing I've experienced with the Travis-CI cache is that
>>> the time spent uploading & downloading the remote cache - in that case of
>>> all the pip installed dependencies - negated the benefits. Probably for
>>> Beam it will have a greater benefit if we can skip most of the build.
>>>
>>> Regarding integration tests in precommit: I think it is OK to run maybe
>>> one Dataflow job in precommit, but it should be in parallel with the unit
>>> tests and just a smoke test that takes 5 minutes, not a suite that takes 35
>>> minutes. So IMO that is low-hanging fruit. If this would make postcommit
>>> unstable, then it also means precommit is unstable. Both are troublesome.
>>>
>>> More short term, some possible hacks:
>>>
>>>  - Point gradle to cache outside the git workspace. We already did this
>>> for .m2 and it helped a lot.
>>>  - Intersect touched files with projects. Our nonstandard project names
>>> might be a pain here. Not sure if fixing that is on the roadmap.
>>>
>>> Kenn
>>>
>>> On Wed, May 23, 2018 at 9:31 AM Ismaël Mejía <ie...@gmail.com> wrote:
>>>
>>>> I second Robert idea of ‘inteligently’ running only the affected tests,
>>>> probably
>>>> there is no need to run Java for a go fix (and eventually if any issue
>>>> it
>>>> can be
>>>> catched in postcommit), same for a dev who just fixed something in
>>>> KafkaIO
>>>> and has
>>>> to wait for other IO tests to pass. I suppose that languages, IOs and
>>>> extensions
>>>> are ‘easy’ to isolate so maybe we can start with those.
>>>>
>>>> Earlier signals are also definitely great to have too, but not sure how
>>>> we
>>>> can
>>>> have those with the current infra.
>>>>
>>>>  From a quicklook the biggest time is consumed by the examples module
>>>> probably
>>>> because they run in Dataflow with real IOs no?, that module alone takes
>>>> ~35
>>>> minutes, so maybe moving it to postcommit will gain us some quick
>>>> improvement.
>>>> On the other hand we should probably not dismiss the consequences of
>>>> moving
>>>> more
>>>> stuff to postcommit given that our current postcommit is not the most
>>>> stable, or
>>>> the quickest, only the Dataflow suite takes 1h30!
>>>>
>>>>
>>>> On Tue, May 22, 2018 at 12:01 AM Mikhail Gryzykhin <mi...@google.com>
>>>> wrote:
>>>>
>>>> > What we can do here is estimate how much effort we want to put in and
>>>> set
>>>> remote target.
>>>> > Such as:
>>>> > Third quarter 2018 -- 1hr SLO
>>>> > Forth quarter 2018 -- 30min SLO,
>>>> > etc.
>>>>
>>>> > Combined with policy for newly added tests, this can give us some
>>>> goal to
>>>> aim for.
>>>>
>>>> > --Mikhail
>>>>
>>>> > Have feedback?
>>>>
>>>>
>>>> > On Mon, May 21, 2018 at 2:06 PM Scott Wegner <sw...@google.com>
>>>> wrote:
>>>>
>>>> >> Thanks for the proposal, I left comments in the doc. Overall I think
>>>> it's a great idea.
>>>>
>>>> >> I've seen other projects with much faster pre-commits, and it
>>>> requires
>>>> strict guidelines on unit test design and keeping tests isolated
>>>> in-memory
>>>> as much as possible. That's not currently the case in Java; we have
>>>> pre-commits which submit pipelines to Dataflow service.
>>>>
>>>> >> I don't know if it's feasible to get Java down to 15-20 mins in the
>>>> short term, but a good starting point would be to document the
>>>> requirements
>>>> for a test to run as pre-commit, and start enforcing it for new tests.
>>>>
>>>>
>>>> >> On Fri, May 18, 2018 at 3:25 PM Henning Rohde <he...@google.com>
>>>> wrote:
>>>>
>>>> >>> Good proposal. I think it should be considered in tandem with the
>>>> "No
>>>> commit on red post-commit" proposal and could be far more ambitious
>>>> than 2
>>>> hours. For example, something in the <15-20 mins range, say, would be
>>>> much
>>>> less of an inconvenience to the development effort. Go takes ~3 mins,
>>>> which
>>>> means that it is practical to wait until a PR is green before asking
>>>> anyone
>>>> to look at it. If I need to wait for a Java or Python pre-commit, I task
>>>> switch and come back later. If the post-commits are enforced to be
>>>> green,
>>>> we could possibly gain a much more productive flow at the cost of the
>>>> occasional post-commit break, compared to now. Maybe IOs can be less
>>>> extensively tested pre-commit, for example, or only if actually changed?
>>>>
>>>> >>> I also like Robert's suggestion of spitting up pre-commits into
>>>> something more fine-grained to get a clear partial signal quicker. If we
>>>> have an adequate number of Jenkins slots, it might also speed things up
>>>> overall.
>>>>
>>>> >>> Thanks,
>>>> >>>    Henning
>>>>
>>>> >>> On Fri, May 18, 2018 at 12:30 PM Scott Wegner <sw...@google.com>
>>>> wrote:
>>>>
>>>> >>>> re: intelligently skipping tests for code that doesn't change (i.e.
>>>> Java tests on Python PR): this should be possible. We already have
>>>> build-caching enabled in Gradle, but I believe it is local to the git
>>>> workspace and doesn't persist between Jenkins runs.
>>>>
>>>> >>>> With a quick search, I see there is a Jenkins Build Cacher Plugin
>>>> [1]
>>>> that hooks into Gradle build cache and does exactly what we need. Does
>>>> anybody know whether we could get this enabled on our Jenkins?
>>>>
>>>> >>>> [1] https://wiki.jenkins.io/display/JENKINS/Job+Cacher+Plugin
>>>>
>>>> >>>> On Fri, May 18, 2018 at 12:08 PM Robert Bradshaw <
>>>> robertwb@google.com>
>>>> wrote:
>>>>
>>>> >>>>> [somehow  my email got garbled...]
>>>>
>>>> >>>>> Now that we're using gradle, perhaps we could be more intelligent
>>>> about only running the affected tests? E.g. when you touch Python (or
>>>> Go)
>>>> you shouldn't need to run the Java precommit at all, which would reduce
>>>> the
>>>> latency for those PRs and also the time spent in queue. Presumably this
>>>> could even be applied per-module for the Java tests. (Maybe a large,
>>>> shared
>>>> build cache could help here as well...)
>>>>
>>>> >>>>> I also wouldn't be opposed to a quicker immediate signal, plus
>>>> more
>>>> extensive tests before actually merging. It's also nice to not have to
>>>> wait
>>>> an hour to see that you have a lint error; quick stuff like that could
>>>> be
>>>> signaled quickly before a contributor looses context.
>>>>
>>>> >>>>> - Robert
>>>>
>>>>
>>>>
>>>> >>>>> On Fri, May 18, 2018 at 5:55 AM Kenneth Knowles <kl...@google.com>
>>>> wrote:
>>>>
>>>> >>>>>> I like the idea. I think it is a good time for the project to
>>>> start
>>>> tracking this and keeping it usable.
>>>>
>>>> >>>>>> Certainly 2 hours is more than enough, is that not so? The Java
>>>> precommit seems to take <=40 minutes while Python takes ~20 and Go is so
>>>> fast it doesn't matter. Do we have enough stragglers that we don't make
>>>> it
>>>> in the 95th percentile? Is the time spent in the Jenkins queue?
>>>>
>>>> >>>>>> For our current coverage, I'd be willing to go for:
>>>>
>>>> >>>>>>    - 1 hr hard cap (someone better at stats could choose %ile)
>>>> >>>>>>    - roll back or remove test from precommit if fix looks like
>>>> more
>>>> than 1 week (roll back if it is perf degradation, remove test from
>>>> precommit if it is additional coverage that just doesn't fit in the
>>>> time)
>>>>
>>>> >>>>>> There's a longer-term issue that doing a full build each time is
>>>> expected to linearly scale up with the size of our repo (it is the
>>>> monorepo
>>>> problem but for a minirepo) so there is no cap that is feasible until we
>>>> have effective cross-build caching. And my long-term goal would be <30
>>>> minutes. At the latency of opening a pull request and then checking your
>>>> email that's not burdensome, but an hour is.
>>>>
>>>> >>>>>> Kenn
>>>>
>>>> >>>>>> On Thu, May 17, 2018 at 6:54 PM Udi Meiri <eh...@google.com>
>>>> wrote:
>>>>
>>>> >>>>>>> HI,
>>>> >>>>>>> I have a proposal to improve contributor experience by keeping
>>>> precommit times low.
>>>>
>>>> >>>>>>> I'm looking to get community consensus and approval about:
>>>> >>>>>>> 1. How long should precommits take. 2 hours @95th percentile
>>>> over
>>>> the past 4 weeks is the current proposal.
>>>> >>>>>>> 2. The process for dealing with slowness. Do we: fix, roll back,
>>>> remove a test from precommit?
>>>> >>>>>>> Rolling back if a fix is estimated to take longer than 2 weeks
>>>> is
>>>> the current proposal.
>>>>
>>>>
>>>>
>>>> https://docs.google.com/document/d/1udtvggmS2LTMmdwjEtZCcUQy6aQAiYTI3OrTP8CLfJM/edit?usp=sharing
>>>>
>>>