You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@beam.apache.org by Ahmet Altay <al...@google.com> on 2019/11/04 20:05:53 UTC

Re: Python Precommit duration pushing 2 hours

Python precommits are still timing out on #9925. I am guessing that means
this change would not be enough.

I am proposing cutting down the number of test variants we run in
precommits. Currently for each version we ran the following variants
serially:
- base: Runs all unit tests with tox
- Cython: Installs cython and runs all unit tests as base version. The
original purpose was to ensure that tests pass with or without cython.
There is probably a huge overlap with base. (IIRC only a few coders have
different slow vs fast tests.)
- GCP: Installs GCP dependencies and tests all base + additional gcp
specific tests. The original purpose was to ensure that GCP is an optional
component and all non-GCP tests still works without GCP components.

We can reduce the list to cython + GCP tests only. This will cover the same
group of tests and will check that tests pass with or without cython or GCP
dependencies. This could reduce the precommit time by ~30 minutes.

What do you think?

Ahmet


On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com>
wrote:

> https://github.com/apache/beam/pull/9925
>
> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
> >
> > I don't have the bandwidth right now to tackle this. Feel free to take
> it.
> >
> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> The Python SDK does as well. These calls are coming from
> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which are
> >> invoked once per pipene or bundle. They are, however, surprisingly
> >> expensive. Even memoizing across those three calls should save a
> >> significant amount of time. Udi, did you want to tackle this?
> >>
> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
> >> times in this test, and [Applied]PTransform.to_fn_api being called
> >> 3111 times, so that in itself might be interesting to investigate.
> >>
> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com>
> wrote:
> >> >
> >> > As does the Go SDK. Invokers are memoized and when possible code is
> generated to avoid reflection.
> >> >
> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com> wrote:
> >> >>
> >> >> Noting for the benefit of the thread archive in case someone goes
> digging and wonders if this affects other SDKs: the Java SDK memoizes
> DoFnSignatures and generated DoFnInvoker classes.
> >> >>
> >> >> Kenn
> >> >>
> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com> wrote:
> >> >>>
> >> >>> Re: #9283 slowing down tests, ideas for slowness:
> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
> >> >>> 2. The PR somehow changed how coders are selected, and now we're
> using less efficient ones.
> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
> >> >>>
> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and 3.7
> and got these cool graphs (attached).
> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time
> (IIUC), gets called ~230k times
> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k times
> >> >>>
> >> >>> After memoization of get_function_args_defaults:
> >> >>> 2.7: core:294:get_function_arguments 20.02%
> >> >>> 3.7: core:294:get_function_arguments 8.11%
> >> >>>
> >> >>>
> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <pa...@google.com>
> wrote:
> >> >>>>
> >> >>>> *not deciles, but 9-percentiles : )
> >> >>>>
> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <pa...@google.com>
> wrote:
> >> >>>>>
> >> >>>>> I've ran the tests in Python 2 (without cython), and used a
> utility to track runtime for each test method. I found some of the
> following things:
> >> >>>>> - Total test methods run: 2665
> >> >>>>> - Total test runtime: 990 seconds
> >> >>>>> - Deciles of time spent:
> >> >>>>>   - 1949 tests run in the first 9% of time
> >> >>>>>   - 173 in the 9-18% rang3e
> >> >>>>>   - 130 in the 18-27% range
> >> >>>>>   - 95 in the 27-36% range
> >> >>>>>   - 77
> >> >>>>>   - 66
> >> >>>>>   - 55
> >> >>>>>   - 46
> >> >>>>>   - 37
> >> >>>>>   - 24
> >> >>>>>   - 13 tests run in the last 9% of time. This represents about 1
> minute and a half.
> >> >>>>>
> >> >>>>> We may be able to look at the slowest X tests, and get gradual
> improvements from there. Although it seems .. not dramatic ones : )
> >> >>>>>
> >> >>>>> FWIW I uploaded the results here:
> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
> >> >>>>>
> >> >>>>> The slowest 13 tests were:
> >> >>>>>
> >> >>>>>
> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
> >> >>>>>   5.253582000732422),
> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
> >> >>>>>   7.907713890075684),
> >> >>>>>
> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
> >> >>>>>   5.237942934036255),
> >> >>>>>
> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
> >> >>>>>   5.563946008682251),
> >> >>>>>
> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
> >> >>>>>   5.680700063705444),
> >> >>>>>
> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
> >> >>>>>   6.111238956451416),
> >> >>>>>
> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
> >> >>>>>   6.007534980773926),
> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
> >> >>>>>   13.993916988372803),
> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
> >> >>>>>   6.3383049964904785),
> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
> >> >>>>>   9.157485008239746),
> >> >>>>>
> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
> >> >>>>>   5.191173076629639),
> >> >>>>>
> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
> >> >>>>>   6.2221620082855225),
> >> >>>>>
> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
> >> >>>>>   7.7187910079956055)]
> >> >>>>>
> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <pa...@google.com>
> wrote:
> >> >>>>>>
> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to
> reduce FnApiRunnerTest variations.
> >> >>>>>> I'm not in a rush to merge, but rather happy to start a
> discussion.
> >> >>>>>> I'll also try to figure out if there are other tests slowing
> down the suite significantly.
> >> >>>>>> Best
> >> >>>>>> -P.
> >> >>>>>>
> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
> valentyn@google.com> wrote:
> >> >>>>>>>
> >> >>>>>>> Thanks, Brian.
> >> >>>>>>> +Udi Meiri
> >> >>>>>>> As next step, it would be good to know whether slowdown is
> caused by tests in this PR, or its effect on other tests, and to confirm
> that only Python 2 codepaths were affected.
> >> >>>>>>>
> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
> bhulette@google.com> wrote:
> >> >>>>>>>>
> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
> and 9/15 to see if I could find the source of the spike that happened
> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
> search would reveal some mis-guided configuration change, but as far as I
> can tell 9283 just added a well-tested feature. I don't think there's
> anything to learn from that... I just wanted to circle back about it in
> case others are curious about that spike.
> >> >>>>>>>>
> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
> >> >>>>>>>>
> >> >>>>>>>> Brian
> >> >>>>>>>>
> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
> >> >>>>>>>>
> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >>>>>>>>>
> >> >>>>>>>>> I think it makes sense to remove some of the extra
> FnApiRunner configurations. Perhaps some of the multiworkers and some of
> the grpc versions?
> >> >>>>>>>>> Best
> >> >>>>>>>>> -P.
> >> >>>>>>>>>
> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
> robertwb@google.com> wrote:
> >> >>>>>>>>>>
> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive,
> taking 10-15+
> >> >>>>>>>>>> minutes on each version of Python. This test consists of a
> base class
> >> >>>>>>>>>> that is basically a validates runner suite, and is then run
> in several
> >> >>>>>>>>>> configurations, many more of which (including some expensive
> ones)
> >> >>>>>>>>>> have been added lately.
> >> >>>>>>>>>>
> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
> >> >>>>>>>>>> class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
> >> >>>>>>>>>> class
> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
> >> >>>>>>>>>>
> >> >>>>>>>>>> I'm not convinced we need to run all of these permutations,
> or at
> >> >>>>>>>>>> least not all tests in all permutations.
> >> >>>>>>>>>>
> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
> >> >>>>>>>>>> <va...@google.com> wrote:
> >> >>>>>>>>>> >
> >> >>>>>>>>>> > I took another look at this and precommit ITs are already
> running in parallel, albeit in the same suite. However it appears Python
> precommits became slower, especially Python 2 precommits [35 min per suite
> x 3 suites], see [1]. Not sure yet what caused the increase, but precommits
> used to be faster. Perhaps we have added a slow test or a lot of new tests.
> >> >>>>>>>>>> >
> >> >>>>>>>>>> > [1]
> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
> >> >>>>>>>>>> >
> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
> altay@google.com> wrote:
> >> >>>>>>>>>> >>
> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite sounds
> good. Anyone is interested in doing that?
> >> >>>>>>>>>> >>
> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <
> valentyn@google.com> wrote:
> >> >>>>>>>>>> >>>
> >> >>>>>>>>>> >>> This should not increase the queue time substantially,
> since precommit ITs are running sequentially with precommit tests, unlike
> multiple precommit tests which run in parallel to each other.
> >> >>>>>>>>>> >>>
> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming
> wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
> >> >>>>>>>>>> >>>
> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
> altay@google.com> wrote:
> >> >>>>>>>>>> >>>>
> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would be,
> when Chad tried to do something similar [1] it was noted that the total
> time to run all precommit tests would increase and also potentially
> increase the queue time.
> >> >>>>>>>>>> >>>>
> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of IT
> tests in precommits and run the whole suite as part of post commit tests.
> >> >>>>>>>>>> >>>>
> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
> >> >>>>>>>>>> >>>>
> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn Tymofieiev <
> valentyn@google.com> wrote:
> >> >>>>>>>>>> >>>>>
> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT tests
> into a separate suite from precommit tests, and run it in parallel.
> >> >>>>>>>>>> >>>>>
> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <
> bhulette@google.com> wrote:
> >> >>>>>>>>>> >>>>>>
> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now [1].
> Just visually it looks like the average length is 1.5h or so, but it spikes
> up to 2h. I've had several precommit runs get aborted due to the 2 hour
> limit.
> >> >>>>>>>>>> >>>>>>
> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back on
> 9/6 and the duration has been steadily rising since then. Is there anything
> we can do about this?
> >> >>>>>>>>>> >>>>>>
> >> >>>>>>>>>> >>>>>> Brian
> >> >>>>>>>>>> >>>>>>
> >> >>>>>>>>>> >>>>>> [1]
> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>

Re: Python Precommit duration pushing 2 hours

Posted by Robert Bradshaw <ro...@google.com>.
On Thu, Nov 14, 2019 at 2:58 PM Ahmet Altay <al...@google.com> wrote:
>
> On Thu, Nov 14, 2019 at 2:55 PM Mikhail Gryzykhin <mi...@google.com> wrote:
>>
>> Hi Everyone,
>>
>> Python precommit phrase timeouts for (roughly) 80% of the jobs in 2 hours. This also blocks release branch validation. I suggest to bump the timeout to 3 hours while we are working on a proper solution. This way many people can get unblocked.
>>
>> I believe the change can be rather small: https://github.com/apache/beam/pull/10121
>
> +1 to unblock the current state.

+1 from me as well. Who (if anyone) is looking into the root cause? I
don't want to bump this up and forget it--2 hours is painful enough as
is.

>> On Mon, Nov 11, 2019 at 5:24 PM Ning Kang <ni...@google.com> wrote:
>>>
>>> I'm removing the additional interactive test env + suite and add [interactive] dependencies as extra dependencies in tests_require: https://github.com/apache/beam/pull/10068
>>>
>>> On Mon, Nov 11, 2019 at 2:15 PM Robert Bradshaw <ro...@google.com> wrote:
>>>>
>>>> On Fri, Nov 8, 2019 at 5:45 PM Ahmet Altay <al...@google.com> wrote:
>>>> >
>>>> > I looked at the log but I could not figure what is causing the timeout because the gradle scan links are missing. I sampled a few of the successful jobs, It seems like python 3.7 and python 2 are running 3 tests in serial {interactive, py37cython, py37gcp} and {docs, py27cython, py27gcp} respectively. These two versions are pushing the total time because other variants are now only running {cython, gcp} versions.
>>>> >
>>>> > I suggest breaking up docs, and interactive into 2 separate suites of their own. docs is actually faster than interactive,just separating that out to a new suite might help.
>>>> >
>>>> > Interactive was recently added (https://github.com/apache/beam/pull/9741). +Ning Kang could you separate interactive to new suite?
>>>>
>>>> I would ask why interactive is a separate tox configuration at all; I
>>>> don't think there's a need to run every test again with a couple of
>>>> extra dependencies (adding ~30 minutes to every presumbit). I think it
>>>> would be much more valuable to run the (presumably relatively small)
>>>> set of interactive tests in all modes.
>>>>
>>>> (The other suites are to guerentee the tests specifically run
>>>> *without* installing gcp and *without* compiling with Cython.)
>>>>
>>>> > On Fri, Nov 8, 2019 at 11:09 AM Robert Bradshaw <ro...@google.com> wrote:
>>>> >>
>>>> >> Just saw another 2-hour timeout:
>>>> >> https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
>>>> >> perhaps we're not out of the woods yet (though in general things have
>>>> >> been a lot better).
>>>> >>
>>>> >> On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
>>>> >> >
>>>> >> > GCP tests are already on separate locations. IO related tests are under /sdks/python/apache_beam/io/gcp and Dataflow related tests are under sdks/python/apache_beam/runners/dataflow. It should be a matter of changing gradle files to run either one of the base tests or GCP tests depending on the types of changes. I do not expect this to have any material impact on the precommit times because these two test suites take about exactly the same time to complete.
>>>> >> >
>>>> >> > #9985 is merged now. Precommit times on master branch dropped to ~1h 20 for the last 5 runs.
>>>> >> >
>>>> >> > On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com> wrote:
>>>> >> >>
>>>> >> >> +1 to moving the GCP tests outside of core. If there are issues that only show up on GCP tests but not in core, it might be an indication that there needs to be another test in core covering that, but I think that should be pretty rare.
>>>> >> >>
>>>> >> >> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org> wrote:
>>>> >> >>>
>>>> >> >>> +1 to moving forward with this
>>>> >> >>>
>>>> >> >>> Could we move GCP tests outside the core? Then only code changes touches/affecting GCP would cause them to run in precommit. Could still run them in postcommit in their own suite. If the core has reasonably stable abstractions that the connectors are built on, this should not change coverage much.
>>>> >> >>>
>>>> >> >>> Kenn
>>>> >> >>>
>>>> >> >>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:
>>>> >> >>>>
>>>> >> >>>> PR for the proposed change: https://github.com/apache/beam/pull/9985
>>>> >> >>>>
>>>> >> >>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
>>>> >> >>>>>
>>>> >> >>>>> +1
>>>> >> >>>>>
>>>> >> >>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com> wrote:
>>>> >> >>>>>>
>>>> >> >>>>>> +1, this seems like a good step with a clear win.
>>>> >> >>>>>>
>>>> >> >>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>>>> >> >>>>>> >
>>>> >> >>>>>> > Python precommits are still timing out on #9925. I am guessing that means this change would not be enough.
>>>> >> >>>>>> >
>>>> >> >>>>>> > I am proposing cutting down the number of test variants we run in precommits. Currently for each version we ran the following variants serially:
>>>> >> >>>>>> > - base: Runs all unit tests with tox
>>>> >> >>>>>> > - Cython: Installs cython and runs all unit tests as base version. The original purpose was to ensure that tests pass with or without cython. There is probably a huge overlap with base. (IIRC only a few coders have different slow vs fast tests.)
>>>> >> >>>>>> > - GCP: Installs GCP dependencies and tests all base + additional gcp specific tests. The original purpose was to ensure that GCP is an optional component and all non-GCP tests still works without GCP components.
>>>> >> >>>>>> >
>>>> >> >>>>>> > We can reduce the list to cython + GCP tests only. This will cover the same group of tests and will check that tests pass with or without cython or GCP dependencies. This could reduce the precommit time by ~30 minutes.
>>>> >> >>>>>> >
>>>> >> >>>>>> > What do you think?
>>>> >> >>>>>> >
>>>> >> >>>>>> > Ahmet
>>>> >> >>>>>> >
>>>> >> >>>>>> >
>>>> >> >>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com> wrote:
>>>> >> >>>>>> >>
>>>> >> >>>>>> >> https://github.com/apache/beam/pull/9925
>>>> >> >>>>>> >>
>>>> >> >>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>>>> >> >>>>>> >> >
>>>> >> >>>>>> >> > I don't have the bandwidth right now to tackle this. Feel free to take it.
>>>> >> >>>>>> >> >
>>>> >> >>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com> wrote:
>>>> >> >>>>>> >> >>
>>>> >> >>>>>> >> >> The Python SDK does as well. These calls are coming from
>>>> >> >>>>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which are
>>>> >> >>>>>> >> >> invoked once per pipene or bundle. They are, however, surprisingly
>>>> >> >>>>>> >> >> expensive. Even memoizing across those three calls should save a
>>>> >> >>>>>> >> >> significant amount of time. Udi, did you want to tackle this?
>>>> >> >>>>>> >> >>
>>>> >> >>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
>>>> >> >>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
>>>> >> >>>>>> >> >> 3111 times, so that in itself might be interesting to investigate.
>>>> >> >>>>>> >> >>
>>>> >> >>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com> wrote:
>>>> >> >>>>>> >> >> >
>>>> >> >>>>>> >> >> > As does the Go SDK. Invokers are memoized and when possible code is generated to avoid reflection.
>>>> >> >>>>>> >> >> >
>>>> >> >>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com> wrote:
>>>> >> >>>>>> >> >> >>
>>>> >> >>>>>> >> >> >> Noting for the benefit of the thread archive in case someone goes digging and wonders if this affects other SDKs: the Java SDK memoizes DoFnSignatures and generated DoFnInvoker classes.
>>>> >> >>>>>> >> >> >>
>>>> >> >>>>>> >> >> >> Kenn
>>>> >> >>>>>> >> >> >>
>>>> >> >>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>
>>>> >> >>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>>>> >> >>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
>>>> >> >>>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and now we're using less efficient ones.
>>>> >> >>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>>>> >> >>>>>> >> >> >>>
>>>> >> >>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and 3.7 and got these cool graphs (attached).
>>>> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time (IIUC), gets called ~230k times
>>>> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k times
>>>> >> >>>>>> >> >> >>>
>>>> >> >>>>>> >> >> >>> After memoization of get_function_args_defaults:
>>>> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>>>> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>>>> >> >>>>>> >> >> >>>
>>>> >> >>>>>> >> >> >>>
>>>> >> >>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <pa...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>
>>>> >> >>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>>>> >> >>>>>> >> >> >>>>
>>>> >> >>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <pa...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>
>>>> >> >>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a utility to track runtime for each test method. I found some of the following things:
>>>> >> >>>>>> >> >> >>>>> - Total test methods run: 2665
>>>> >> >>>>>> >> >> >>>>> - Total test runtime: 990 seconds
>>>> >> >>>>>> >> >> >>>>> - Deciles of time spent:
>>>> >> >>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>>>> >> >>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>>>> >> >>>>>> >> >> >>>>>   - 130 in the 18-27% range
>>>> >> >>>>>> >> >> >>>>>   - 95 in the 27-36% range
>>>> >> >>>>>> >> >> >>>>>   - 77
>>>> >> >>>>>> >> >> >>>>>   - 66
>>>> >> >>>>>> >> >> >>>>>   - 55
>>>> >> >>>>>> >> >> >>>>>   - 46
>>>> >> >>>>>> >> >> >>>>>   - 37
>>>> >> >>>>>> >> >> >>>>>   - 24
>>>> >> >>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents about 1 minute and a half.
>>>> >> >>>>>> >> >> >>>>>
>>>> >> >>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and get gradual improvements from there. Although it seems .. not dramatic ones : )
>>>> >> >>>>>> >> >> >>>>>
>>>> >> >>>>>> >> >> >>>>> FWIW I uploaded the results here: https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>>>> >> >>>>>> >> >> >>>>>
>>>> >> >>>>>> >> >> >>>>> The slowest 13 tests were:
>>>> >> >>>>>> >> >> >>>>>
>>>> >> >>>>>> >> >> >>>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>>>> >> >>>>>> >> >> >>>>>   5.253582000732422),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>>>> >> >>>>>> >> >> >>>>>   7.907713890075684),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>>>> >> >>>>>> >> >> >>>>>   5.237942934036255),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>>>> >> >>>>>> >> >> >>>>>   5.563946008682251),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>>>> >> >>>>>> >> >> >>>>>   5.680700063705444),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>>>> >> >>>>>> >> >> >>>>>   6.111238956451416),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>>>> >> >>>>>> >> >> >>>>>   6.007534980773926),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>>>> >> >>>>>> >> >> >>>>>   13.993916988372803),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>>>> >> >>>>>> >> >> >>>>>   6.3383049964904785),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>>>> >> >>>>>> >> >> >>>>>   9.157485008239746),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>>>> >> >>>>>> >> >> >>>>>   5.191173076629639),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>>>> >> >>>>>> >> >> >>>>>   6.2221620082855225),
>>>> >> >>>>>> >> >> >>>>>  ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>>>> >> >>>>>> >> >> >>>>>   7.7187910079956055)]
>>>> >> >>>>>> >> >> >>>>>
>>>> >> >>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <pa...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>
>>>> >> >>>>>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest variations.
>>>> >> >>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a discussion.
>>>> >> >>>>>> >> >> >>>>>> I'll also try to figure out if there are other tests slowing down the suite significantly.
>>>> >> >>>>>> >> >> >>>>>> Best
>>>> >> >>>>>> >> >> >>>>>> -P.
>>>> >> >>>>>> >> >> >>>>>>
>>>> >> >>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>
>>>> >> >>>>>> >> >> >>>>>>> Thanks, Brian.
>>>> >> >>>>>> >> >> >>>>>>> +Udi Meiri
>>>> >> >>>>>> >> >> >>>>>>> As next step, it would be good to know whether slowdown is caused by tests in this PR, or its effect on other tests, and to confirm that only Python 2 codepaths were affected.
>>>> >> >>>>>> >> >> >>>>>>>
>>>> >> >>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <bh...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1 and 9/15 to see if I could find the source of the spike that happened around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this search would reveal some mis-guided configuration change, but as far as I can tell 9283 just added a well-tested feature. I don't think there's anything to learn from that... I just wanted to circle back about it in case others are curious about that spike.
>>>> >> >>>>>> >> >> >>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>>>> >> >>>>>> >> >> >>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>> Brian
>>>> >> >>>>>> >> >> >>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>>>> >> >>>>>> >> >> >>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <pa...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra FnApiRunner configurations. Perhaps some of the multiworkers and some of the grpc versions?
>>>> >> >>>>>> >> >> >>>>>>>>> Best
>>>> >> >>>>>> >> >> >>>>>>>>> -P.
>>>> >> >>>>>> >> >> >>>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <ro...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive, taking 10-15+
>>>> >> >>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists of a base class
>>>> >> >>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is then run in several
>>>> >> >>>>>> >> >> >>>>>>>>>> configurations, many more of which (including some expensive ones)
>>>> >> >>>>>> >> >> >>>>>>>>>> have been added lately.
>>>> >> >>>>>> >> >> >>>>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>>>> >> >>>>>> >> >> >>>>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these permutations, or at
>>>> >> >>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>>>> >> >>>>>> >> >> >>>>>>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>>>> >> >>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>> >
>>>> >> >>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are already running in parallel, albeit in the same suite. However it appears Python precommits became slower, especially Python 2 precommits [35 min per suite x 3 suites], see [1]. Not sure yet what caused the increase, but precommits used to be faster. Perhaps we have added a slow test or a lot of new tests.
>>>> >> >>>>>> >> >> >>>>>>>>>> >
>>>> >> >>>>>> >> >> >>>>>>>>>> > [1] https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>>>> >> >>>>>> >> >> >>>>>>>>>> >
>>>> >> >>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <al...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>> >>
>>>> >> >>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite sounds good. Anyone is interested in doing that?
>>>> >> >>>>>> >> >> >>>>>>>>>> >>
>>>> >> >>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time substantially, since precommit ITs are running sequentially with precommit tests, unlike multiple precommit tests which run in parallel to each other.
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <al...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would be, when Chad tried to do something similar [1] it was noted that the total time to run all precommit tests would increase and also potentially increase the queue time.
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of IT tests in precommits and run the whole suite as part of post commit tests.
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn Tymofieiev <va...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT tests into a separate suite from precommit tests, and run it in parallel.
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <bh...@google.com> wrote:
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now [1]. Just visually it looks like the average length is 1.5h or so, but it spikes up to 2h. I've had several precommit runs get aborted due to the 2 hour limit.
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back on 9/6 and the duration has been steadily rising since then. Is there anything we can do about this?
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Brian
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> [1] http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4

Re: Python Precommit duration pushing 2 hours

Posted by Ahmet Altay <al...@google.com>.
On Thu, Nov 14, 2019 at 2:55 PM Mikhail Gryzykhin <mi...@google.com> wrote:

> Hi Everyone,
>
> Python precommit phrase timeouts for (roughly) 80% of the jobs in 2 hours.
> This also blocks release branch validation. I suggest to bump the timeout
> to 3 hours while we are working on a proper solution. This way many people
> can get unblocked.
>
> I believe the change can be rather small:
> https://github.com/apache/beam/pull/10121
>

+1 to unblock the current state.


>
> --Mikhail
>
>
>
> On Mon, Nov 11, 2019 at 5:24 PM Ning Kang <ni...@google.com> wrote:
>
>> I'm removing the additional interactive test env + suite and add
>> [interactive] dependencies as extra dependencies in tests_require:
>> https://github.com/apache/beam/pull/10068
>>
>> On Mon, Nov 11, 2019 at 2:15 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> On Fri, Nov 8, 2019 at 5:45 PM Ahmet Altay <al...@google.com> wrote:
>>> >
>>> > I looked at the log but I could not figure what is causing the timeout
>>> because the gradle scan links are missing. I sampled a few of the
>>> successful jobs, It seems like python 3.7 and python 2 are running 3 tests
>>> in serial {interactive, py37cython, py37gcp} and {docs, py27cython,
>>> py27gcp} respectively. These two versions are pushing the total time
>>> because other variants are now only running {cython, gcp} versions.
>>> >
>>> > I suggest breaking up docs, and interactive into 2 separate suites of
>>> their own. docs is actually faster than interactive,just separating that
>>> out to a new suite might help.
>>> >
>>> > Interactive was recently added (
>>> https://github.com/apache/beam/pull/9741). +Ning Kang could you
>>> separate interactive to new suite?
>>>
>>> I would ask why interactive is a separate tox configuration at all; I
>>> don't think there's a need to run every test again with a couple of
>>> extra dependencies (adding ~30 minutes to every presumbit). I think it
>>> would be much more valuable to run the (presumably relatively small)
>>> set of interactive tests in all modes.
>>>
>>> (The other suites are to guerentee the tests specifically run
>>> *without* installing gcp and *without* compiling with Cython.)
>>>
>>> > On Fri, Nov 8, 2019 at 11:09 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >>
>>> >> Just saw another 2-hour timeout:
>>> >> https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
>>> >> perhaps we're not out of the woods yet (though in general things have
>>> >> been a lot better).
>>> >>
>>> >> On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
>>> >> >
>>> >> > GCP tests are already on separate locations. IO related tests are
>>> under /sdks/python/apache_beam/io/gcp and Dataflow related tests are under
>>> sdks/python/apache_beam/runners/dataflow. It should be a matter of changing
>>> gradle files to run either one of the base tests or GCP tests depending on
>>> the types of changes. I do not expect this to have any material impact on
>>> the precommit times because these two test suites take about exactly the
>>> same time to complete.
>>> >> >
>>> >> > #9985 is merged now. Precommit times on master branch dropped to
>>> ~1h 20 for the last 5 runs.
>>> >> >
>>> >> > On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com>
>>> wrote:
>>> >> >>
>>> >> >> +1 to moving the GCP tests outside of core. If there are issues
>>> that only show up on GCP tests but not in core, it might be an indication
>>> that there needs to be another test in core covering that, but I think that
>>> should be pretty rare.
>>> >> >>
>>> >> >> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org>
>>> wrote:
>>> >> >>>
>>> >> >>> +1 to moving forward with this
>>> >> >>>
>>> >> >>> Could we move GCP tests outside the core? Then only code changes
>>> touches/affecting GCP would cause them to run in precommit. Could still run
>>> them in postcommit in their own suite. If the core has reasonably stable
>>> abstractions that the connectors are built on, this should not change
>>> coverage much.
>>> >> >>>
>>> >> >>> Kenn
>>> >> >>>
>>> >> >>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com>
>>> wrote:
>>> >> >>>>
>>> >> >>>> PR for the proposed change:
>>> https://github.com/apache/beam/pull/9985
>>> >> >>>>
>>> >> >>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com>
>>> wrote:
>>> >> >>>>>
>>> >> >>>>> +1
>>> >> >>>>>
>>> >> >>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >>>>>>
>>> >> >>>>>> +1, this seems like a good step with a clear win.
>>> >> >>>>>>
>>> >> >>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com>
>>> wrote:
>>> >> >>>>>> >
>>> >> >>>>>> > Python precommits are still timing out on #9925. I am
>>> guessing that means this change would not be enough.
>>> >> >>>>>> >
>>> >> >>>>>> > I am proposing cutting down the number of test variants we
>>> run in precommits. Currently for each version we ran the following variants
>>> serially:
>>> >> >>>>>> > - base: Runs all unit tests with tox
>>> >> >>>>>> > - Cython: Installs cython and runs all unit tests as base
>>> version. The original purpose was to ensure that tests pass with or without
>>> cython. There is probably a huge overlap with base. (IIRC only a few coders
>>> have different slow vs fast tests.)
>>> >> >>>>>> > - GCP: Installs GCP dependencies and tests all base +
>>> additional gcp specific tests. The original purpose was to ensure that GCP
>>> is an optional component and all non-GCP tests still works without GCP
>>> components.
>>> >> >>>>>> >
>>> >> >>>>>> > We can reduce the list to cython + GCP tests only. This will
>>> cover the same group of tests and will check that tests pass with or
>>> without cython or GCP dependencies. This could reduce the precommit time by
>>> ~30 minutes.
>>> >> >>>>>> >
>>> >> >>>>>> > What do you think?
>>> >> >>>>>> >
>>> >> >>>>>> > Ahmet
>>> >> >>>>>> >
>>> >> >>>>>> >
>>> >> >>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >>>>>> >>
>>> >> >>>>>> >> https://github.com/apache/beam/pull/9925
>>> >> >>>>>> >>
>>> >> >>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <
>>> ehudm@google.com> wrote:
>>> >> >>>>>> >> >
>>> >> >>>>>> >> > I don't have the bandwidth right now to tackle this. Feel
>>> free to take it.
>>> >> >>>>>> >> >
>>> >> >>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >>>>>> >> >>
>>> >> >>>>>> >> >> The Python SDK does as well. These calls are coming from
>>> >> >>>>>> >> >> to_runner_api, is_stateful_dofn, and
>>> validate_stateful_dofn which are
>>> >> >>>>>> >> >> invoked once per pipene or bundle. They are, however,
>>> surprisingly
>>> >> >>>>>> >> >> expensive. Even memoizing across those three calls
>>> should save a
>>> >> >>>>>> >> >> significant amount of time. Udi, did you want to tackle
>>> this?
>>> >> >>>>>> >> >>
>>> >> >>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is
>>> being called 30
>>> >> >>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api
>>> being called
>>> >> >>>>>> >> >> 3111 times, so that in itself might be interesting to
>>> investigate.
>>> >> >>>>>> >> >>
>>> >> >>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <
>>> robert@frantil.com> wrote:
>>> >> >>>>>> >> >> >
>>> >> >>>>>> >> >> > As does the Go SDK. Invokers are memoized and when
>>> possible code is generated to avoid reflection.
>>> >> >>>>>> >> >> >
>>> >> >>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <
>>> klk@google.com> wrote:
>>> >> >>>>>> >> >> >>
>>> >> >>>>>> >> >> >> Noting for the benefit of the thread archive in case
>>> someone goes digging and wonders if this affects other SDKs: the Java SDK
>>> memoizes DoFnSignatures and generated DoFnInvoker classes.
>>> >> >>>>>> >> >> >>
>>> >> >>>>>> >> >> >> Kenn
>>> >> >>>>>> >> >> >>
>>> >> >>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <
>>> ehudm@google.com> wrote:
>>> >> >>>>>> >> >> >>>
>>> >> >>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>>> >> >>>>>> >> >> >>> 1. I added a lot of test cases, some with locally
>>> run pipelines.
>>> >> >>>>>> >> >> >>> 2. The PR somehow changed how coders are selected,
>>> and now we're using less efficient ones.
>>> >> >>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down?
>>> (py2 only)
>>> >> >>>>>> >> >> >>>
>>> >> >>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg"
>>> on 2.7 and 3.7 and got these cool graphs (attached).
>>> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of
>>> CPU time (IIUC), gets called ~230k times
>>> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets
>>> called ~200k times
>>> >> >>>>>> >> >> >>>
>>> >> >>>>>> >> >> >>> After memoization of get_function_args_defaults:
>>> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>>> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>>> >> >>>>>> >> >> >>>
>>> >> >>>>>> >> >> >>>
>>> >> >>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >>>>>> >> >> >>>>
>>> >> >>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>>> >> >>>>>> >> >> >>>>
>>> >> >>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>
>>> >> >>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython),
>>> and used a utility to track runtime for each test method. I found some of
>>> the following things:
>>> >> >>>>>> >> >> >>>>> - Total test methods run: 2665
>>> >> >>>>>> >> >> >>>>> - Total test runtime: 990 seconds
>>> >> >>>>>> >> >> >>>>> - Deciles of time spent:
>>> >> >>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>>> >> >>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>>> >> >>>>>> >> >> >>>>>   - 130 in the 18-27% range
>>> >> >>>>>> >> >> >>>>>   - 95 in the 27-36% range
>>> >> >>>>>> >> >> >>>>>   - 77
>>> >> >>>>>> >> >> >>>>>   - 66
>>> >> >>>>>> >> >> >>>>>   - 55
>>> >> >>>>>> >> >> >>>>>   - 46
>>> >> >>>>>> >> >> >>>>>   - 37
>>> >> >>>>>> >> >> >>>>>   - 24
>>> >> >>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This
>>> represents about 1 minute and a half.
>>> >> >>>>>> >> >> >>>>>
>>> >> >>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and
>>> get gradual improvements from there. Although it seems .. not dramatic ones
>>> : )
>>> >> >>>>>> >> >> >>>>>
>>> >> >>>>>> >> >> >>>>> FWIW I uploaded the results here:
>>> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>>> >> >>>>>> >> >> >>>>>
>>> >> >>>>>> >> >> >>>>> The slowest 13 tests were:
>>> >> >>>>>> >> >> >>>>>
>>> >> >>>>>> >> >> >>>>>
>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>>> >> >>>>>> >> >> >>>>>   5.253582000732422),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>>> >> >>>>>> >> >> >>>>>   7.907713890075684),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>>> >> >>>>>> >> >> >>>>>   5.237942934036255),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>>> >> >>>>>> >> >> >>>>>   5.563946008682251),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>>> >> >>>>>> >> >> >>>>>   5.680700063705444),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>>> >> >>>>>> >> >> >>>>>   6.111238956451416),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>>> >> >>>>>> >> >> >>>>>   6.007534980773926),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>>> >> >>>>>> >> >> >>>>>   13.993916988372803),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>>> >> >>>>>> >> >> >>>>>   6.3383049964904785),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>>> >> >>>>>> >> >> >>>>>   9.157485008239746),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>>> >> >>>>>> >> >> >>>>>   5.191173076629639),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>>> >> >>>>>> >> >> >>>>>   6.2221620082855225),
>>> >> >>>>>> >> >> >>>>>
>>> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>>> >> >>>>>> >> >> >>>>>   7.7187910079956055)]
>>> >> >>>>>> >> >> >>>>>
>>> >> >>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>
>>> >> >>>>>> >> >> >>>>>> I have written
>>> https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest
>>> variations.
>>> >> >>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to
>>> start a discussion.
>>> >> >>>>>> >> >> >>>>>> I'll also try to figure out if there are other
>>> tests slowing down the suite significantly.
>>> >> >>>>>> >> >> >>>>>> Best
>>> >> >>>>>> >> >> >>>>>> -P.
>>> >> >>>>>> >> >> >>>>>>
>>> >> >>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn
>>> Tymofieiev <va...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>
>>> >> >>>>>> >> >> >>>>>>> Thanks, Brian.
>>> >> >>>>>> >> >> >>>>>>> +Udi Meiri
>>> >> >>>>>> >> >> >>>>>>> As next step, it would be good to know whether
>>> slowdown is caused by tests in this PR, or its effect on other tests, and
>>> to confirm that only Python 2 codepaths were affected.
>>> >> >>>>>> >> >> >>>>>>>
>>> >> >>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
>>> bhulette@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of
>>> `./gradlew :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits
>>> between 9/1 and 9/15 to see if I could find the source of the spike that
>>> happened around 9/6. It looks like it was due to PR#9283 [1]. I thought
>>> maybe this search would reveal some mis-guided configuration change, but as
>>> far as I can tell 9283 just added a well-tested feature. I don't think
>>> there's anything to learn from that... I just wanted to circle back about
>>> it in case others are curious about that spike.
>>> >> >>>>>> >> >> >>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner
>>> configurations.
>>> >> >>>>>> >> >> >>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>> Brian
>>> >> >>>>>> >> >> >>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>>> >> >>>>>> >> >> >>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the
>>> extra FnApiRunner configurations. Perhaps some of the multiworkers and some
>>> of the grpc versions?
>>> >> >>>>>> >> >> >>>>>>>>> Best
>>> >> >>>>>> >> >> >>>>>>>>> -P.
>>> >> >>>>>> >> >> >>>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert
>>> Bradshaw <ro...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite
>>> expensive, taking 10-15+
>>> >> >>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test
>>> consists of a base class
>>> >> >>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite,
>>> and is then run in several
>>> >> >>>>>> >> >> >>>>>>>>>> configurations, many more of which (including
>>> some expensive ones)
>>> >> >>>>>> >> >> >>>>>>>>>> have been added lately.
>>> >> >>>>>> >> >> >>>>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>>> >> >>>>>> >> >> >>>>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
>>> permutations, or at
>>> >> >>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>>> >> >>>>>> >> >> >>>>>>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn
>>> Tymofieiev
>>> >> >>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>> >
>>> >> >>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit
>>> ITs are already running in parallel, albeit in the same suite. However it
>>> appears Python precommits became slower, especially Python 2 precommits [35
>>> min per suite x 3 suites], see [1]. Not sure yet what caused the increase,
>>> but precommits used to be faster. Perhaps we have added a slow test or a
>>> lot of new tests.
>>> >> >>>>>> >> >> >>>>>>>>>> >
>>> >> >>>>>> >> >> >>>>>>>>>> > [1]
>>> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>>> >> >>>>>> >> >> >>>>>>>>>> >
>>> >> >>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay
>>> <al...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>> >>
>>> >> >>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a
>>> different suite sounds good. Anyone is interested in doing that?
>>> >> >>>>>> >> >> >>>>>>>>>> >>
>>> >> >>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn
>>> Tymofieiev <va...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>> >>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
>>> substantially, since precommit ITs are running sequentially with precommit
>>> tests, unlike multiple precommit tests which run in parallel to each other.
>>> >> >>>>>> >> >> >>>>>>>>>> >>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and
>>> streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of
>>> tests.
>>> >> >>>>>> >> >> >>>>>>>>>> >>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet
>>> Altay <al...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit.
>>> Downside would be, when Chad tried to do something similar [1] it was noted
>>> that the total time to run all precommit tests would increase and also
>>> potentially increase the queue time.
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a
>>> smaller set of IT tests in precommits and run the whole suite as part of
>>> post commit tests.
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>> [1]
>>> https://github.com/apache/beam/pull/9642
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM
>>> Valentyn Tymofieiev <va...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to
>>> Precommit IT tests into a separate suite from precommit tests, and run it
>>> in parallel.
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian
>>> Hulette <bh...@google.com> wrote:
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a
>>> while now [1]. Just visually it looks like the average length is 1.5h or
>>> so, but it spikes up to 2h. I've had several precommit runs get aborted due
>>> to the 2 hour limit.
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up
>>> above 1h back on 9/6 and the duration has been steadily rising since then.
>>> Is there anything we can do about this?
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Brian
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> [1]
>>> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>>>
>>

Re: Python Precommit duration pushing 2 hours

Posted by Mikhail Gryzykhin <mi...@google.com>.
Hi Everyone,

Python precommit phrase timeouts for (roughly) 80% of the jobs in 2 hours.
This also blocks release branch validation. I suggest to bump the timeout
to 3 hours while we are working on a proper solution. This way many people
can get unblocked.

I believe the change can be rather small:
https://github.com/apache/beam/pull/10121

--Mikhail



On Mon, Nov 11, 2019 at 5:24 PM Ning Kang <ni...@google.com> wrote:

> I'm removing the additional interactive test env + suite and add
> [interactive] dependencies as extra dependencies in tests_require:
> https://github.com/apache/beam/pull/10068
>
> On Mon, Nov 11, 2019 at 2:15 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> On Fri, Nov 8, 2019 at 5:45 PM Ahmet Altay <al...@google.com> wrote:
>> >
>> > I looked at the log but I could not figure what is causing the timeout
>> because the gradle scan links are missing. I sampled a few of the
>> successful jobs, It seems like python 3.7 and python 2 are running 3 tests
>> in serial {interactive, py37cython, py37gcp} and {docs, py27cython,
>> py27gcp} respectively. These two versions are pushing the total time
>> because other variants are now only running {cython, gcp} versions.
>> >
>> > I suggest breaking up docs, and interactive into 2 separate suites of
>> their own. docs is actually faster than interactive,just separating that
>> out to a new suite might help.
>> >
>> > Interactive was recently added (
>> https://github.com/apache/beam/pull/9741). +Ning Kang could you separate
>> interactive to new suite?
>>
>> I would ask why interactive is a separate tox configuration at all; I
>> don't think there's a need to run every test again with a couple of
>> extra dependencies (adding ~30 minutes to every presumbit). I think it
>> would be much more valuable to run the (presumably relatively small)
>> set of interactive tests in all modes.
>>
>> (The other suites are to guerentee the tests specifically run
>> *without* installing gcp and *without* compiling with Cython.)
>>
>> > On Fri, Nov 8, 2019 at 11:09 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >>
>> >> Just saw another 2-hour timeout:
>> >> https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
>> >> perhaps we're not out of the woods yet (though in general things have
>> >> been a lot better).
>> >>
>> >> On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
>> >> >
>> >> > GCP tests are already on separate locations. IO related tests are
>> under /sdks/python/apache_beam/io/gcp and Dataflow related tests are under
>> sdks/python/apache_beam/runners/dataflow. It should be a matter of changing
>> gradle files to run either one of the base tests or GCP tests depending on
>> the types of changes. I do not expect this to have any material impact on
>> the precommit times because these two test suites take about exactly the
>> same time to complete.
>> >> >
>> >> > #9985 is merged now. Precommit times on master branch dropped to ~1h
>> 20 for the last 5 runs.
>> >> >
>> >> > On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com>
>> wrote:
>> >> >>
>> >> >> +1 to moving the GCP tests outside of core. If there are issues
>> that only show up on GCP tests but not in core, it might be an indication
>> that there needs to be another test in core covering that, but I think that
>> should be pretty rare.
>> >> >>
>> >> >> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org>
>> wrote:
>> >> >>>
>> >> >>> +1 to moving forward with this
>> >> >>>
>> >> >>> Could we move GCP tests outside the core? Then only code changes
>> touches/affecting GCP would cause them to run in precommit. Could still run
>> them in postcommit in their own suite. If the core has reasonably stable
>> abstractions that the connectors are built on, this should not change
>> coverage much.
>> >> >>>
>> >> >>> Kenn
>> >> >>>
>> >> >>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com>
>> wrote:
>> >> >>>>
>> >> >>>> PR for the proposed change:
>> https://github.com/apache/beam/pull/9985
>> >> >>>>
>> >> >>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com>
>> wrote:
>> >> >>>>>
>> >> >>>>> +1
>> >> >>>>>
>> >> >>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <
>> robertwb@google.com> wrote:
>> >> >>>>>>
>> >> >>>>>> +1, this seems like a good step with a clear win.
>> >> >>>>>>
>> >> >>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com>
>> wrote:
>> >> >>>>>> >
>> >> >>>>>> > Python precommits are still timing out on #9925. I am
>> guessing that means this change would not be enough.
>> >> >>>>>> >
>> >> >>>>>> > I am proposing cutting down the number of test variants we
>> run in precommits. Currently for each version we ran the following variants
>> serially:
>> >> >>>>>> > - base: Runs all unit tests with tox
>> >> >>>>>> > - Cython: Installs cython and runs all unit tests as base
>> version. The original purpose was to ensure that tests pass with or without
>> cython. There is probably a huge overlap with base. (IIRC only a few coders
>> have different slow vs fast tests.)
>> >> >>>>>> > - GCP: Installs GCP dependencies and tests all base +
>> additional gcp specific tests. The original purpose was to ensure that GCP
>> is an optional component and all non-GCP tests still works without GCP
>> components.
>> >> >>>>>> >
>> >> >>>>>> > We can reduce the list to cython + GCP tests only. This will
>> cover the same group of tests and will check that tests pass with or
>> without cython or GCP dependencies. This could reduce the precommit time by
>> ~30 minutes.
>> >> >>>>>> >
>> >> >>>>>> > What do you think?
>> >> >>>>>> >
>> >> >>>>>> > Ahmet
>> >> >>>>>> >
>> >> >>>>>> >
>> >> >>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <
>> robertwb@google.com> wrote:
>> >> >>>>>> >>
>> >> >>>>>> >> https://github.com/apache/beam/pull/9925
>> >> >>>>>> >>
>> >> >>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com>
>> wrote:
>> >> >>>>>> >> >
>> >> >>>>>> >> > I don't have the bandwidth right now to tackle this. Feel
>> free to take it.
>> >> >>>>>> >> >
>> >> >>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
>> robertwb@google.com> wrote:
>> >> >>>>>> >> >>
>> >> >>>>>> >> >> The Python SDK does as well. These calls are coming from
>> >> >>>>>> >> >> to_runner_api, is_stateful_dofn, and
>> validate_stateful_dofn which are
>> >> >>>>>> >> >> invoked once per pipene or bundle. They are, however,
>> surprisingly
>> >> >>>>>> >> >> expensive. Even memoizing across those three calls should
>> save a
>> >> >>>>>> >> >> significant amount of time. Udi, did you want to tackle
>> this?
>> >> >>>>>> >> >>
>> >> >>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being
>> called 30
>> >> >>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api
>> being called
>> >> >>>>>> >> >> 3111 times, so that in itself might be interesting to
>> investigate.
>> >> >>>>>> >> >>
>> >> >>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <
>> robert@frantil.com> wrote:
>> >> >>>>>> >> >> >
>> >> >>>>>> >> >> > As does the Go SDK. Invokers are memoized and when
>> possible code is generated to avoid reflection.
>> >> >>>>>> >> >> >
>> >> >>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <
>> klk@google.com> wrote:
>> >> >>>>>> >> >> >>
>> >> >>>>>> >> >> >> Noting for the benefit of the thread archive in case
>> someone goes digging and wonders if this affects other SDKs: the Java SDK
>> memoizes DoFnSignatures and generated DoFnInvoker classes.
>> >> >>>>>> >> >> >>
>> >> >>>>>> >> >> >> Kenn
>> >> >>>>>> >> >> >>
>> >> >>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <
>> ehudm@google.com> wrote:
>> >> >>>>>> >> >> >>>
>> >> >>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>> >> >>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run
>> pipelines.
>> >> >>>>>> >> >> >>> 2. The PR somehow changed how coders are selected,
>> and now we're using less efficient ones.
>> >> >>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down?
>> (py2 only)
>> >> >>>>>> >> >> >>>
>> >> >>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg"
>> on 2.7 and 3.7 and got these cool graphs (attached).
>> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of
>> CPU time (IIUC), gets called ~230k times
>> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets
>> called ~200k times
>> >> >>>>>> >> >> >>>
>> >> >>>>>> >> >> >>> After memoization of get_function_args_defaults:
>> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>> >> >>>>>> >> >> >>>
>> >> >>>>>> >> >> >>>
>> >> >>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >>>>>> >> >> >>>>
>> >> >>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>> >> >>>>>> >> >> >>>>
>> >> >>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >>>>>> >> >> >>>>>
>> >> >>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython),
>> and used a utility to track runtime for each test method. I found some of
>> the following things:
>> >> >>>>>> >> >> >>>>> - Total test methods run: 2665
>> >> >>>>>> >> >> >>>>> - Total test runtime: 990 seconds
>> >> >>>>>> >> >> >>>>> - Deciles of time spent:
>> >> >>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>> >> >>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>> >> >>>>>> >> >> >>>>>   - 130 in the 18-27% range
>> >> >>>>>> >> >> >>>>>   - 95 in the 27-36% range
>> >> >>>>>> >> >> >>>>>   - 77
>> >> >>>>>> >> >> >>>>>   - 66
>> >> >>>>>> >> >> >>>>>   - 55
>> >> >>>>>> >> >> >>>>>   - 46
>> >> >>>>>> >> >> >>>>>   - 37
>> >> >>>>>> >> >> >>>>>   - 24
>> >> >>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This
>> represents about 1 minute and a half.
>> >> >>>>>> >> >> >>>>>
>> >> >>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and
>> get gradual improvements from there. Although it seems .. not dramatic ones
>> : )
>> >> >>>>>> >> >> >>>>>
>> >> >>>>>> >> >> >>>>> FWIW I uploaded the results here:
>> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>> >> >>>>>> >> >> >>>>>
>> >> >>>>>> >> >> >>>>> The slowest 13 tests were:
>> >> >>>>>> >> >> >>>>>
>> >> >>>>>> >> >> >>>>>
>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>> >> >>>>>> >> >> >>>>>   5.253582000732422),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>> >> >>>>>> >> >> >>>>>   7.907713890075684),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>> >> >>>>>> >> >> >>>>>   5.237942934036255),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>> >> >>>>>> >> >> >>>>>   5.563946008682251),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>> >> >>>>>> >> >> >>>>>   5.680700063705444),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>> >> >>>>>> >> >> >>>>>   6.111238956451416),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>> >> >>>>>> >> >> >>>>>   6.007534980773926),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>> >> >>>>>> >> >> >>>>>   13.993916988372803),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>> >> >>>>>> >> >> >>>>>   6.3383049964904785),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>> >> >>>>>> >> >> >>>>>   9.157485008239746),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>> >> >>>>>> >> >> >>>>>   5.191173076629639),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>> >> >>>>>> >> >> >>>>>   6.2221620082855225),
>> >> >>>>>> >> >> >>>>>
>> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>> >> >>>>>> >> >> >>>>>   7.7187910079956055)]
>> >> >>>>>> >> >> >>>>>
>> >> >>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>
>> >> >>>>>> >> >> >>>>>> I have written
>> https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest
>> variations.
>> >> >>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to
>> start a discussion.
>> >> >>>>>> >> >> >>>>>> I'll also try to figure out if there are other
>> tests slowing down the suite significantly.
>> >> >>>>>> >> >> >>>>>> Best
>> >> >>>>>> >> >> >>>>>> -P.
>> >> >>>>>> >> >> >>>>>>
>> >> >>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn
>> Tymofieiev <va...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>
>> >> >>>>>> >> >> >>>>>>> Thanks, Brian.
>> >> >>>>>> >> >> >>>>>>> +Udi Meiri
>> >> >>>>>> >> >> >>>>>>> As next step, it would be good to know whether
>> slowdown is caused by tests in this PR, or its effect on other tests, and
>> to confirm that only Python 2 codepaths were affected.
>> >> >>>>>> >> >> >>>>>>>
>> >> >>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
>> bhulette@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>
>> >> >>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of
>> `./gradlew :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits
>> between 9/1 and 9/15 to see if I could find the source of the spike that
>> happened around 9/6. It looks like it was due to PR#9283 [1]. I thought
>> maybe this search would reveal some mis-guided configuration change, but as
>> far as I can tell 9283 just added a well-tested feature. I don't think
>> there's anything to learn from that... I just wanted to circle back about
>> it in case others are curious about that spike.
>> >> >>>>>> >> >> >>>>>>>>
>> >> >>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner
>> configurations.
>> >> >>>>>> >> >> >>>>>>>>
>> >> >>>>>> >> >> >>>>>>>> Brian
>> >> >>>>>> >> >> >>>>>>>>
>> >> >>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>> >> >>>>>> >> >> >>>>>>>>
>> >> >>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>
>> >> >>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the
>> extra FnApiRunner configurations. Perhaps some of the multiworkers and some
>> of the grpc versions?
>> >> >>>>>> >> >> >>>>>>>>> Best
>> >> >>>>>> >> >> >>>>>>>>> -P.
>> >> >>>>>> >> >> >>>>>>>>>
>> >> >>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert
>> Bradshaw <ro...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite
>> expensive, taking 10-15+
>> >> >>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test
>> consists of a base class
>> >> >>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite,
>> and is then run in several
>> >> >>>>>> >> >> >>>>>>>>>> configurations, many more of which (including
>> some expensive ones)
>> >> >>>>>> >> >> >>>>>>>>>> have been added lately.
>> >> >>>>>> >> >> >>>>>>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>> >> >>>>>> >> >> >>>>>>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
>> permutations, or at
>> >> >>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>> >> >>>>>> >> >> >>>>>>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn
>> Tymofieiev
>> >> >>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>> >
>> >> >>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit
>> ITs are already running in parallel, albeit in the same suite. However it
>> appears Python precommits became slower, especially Python 2 precommits [35
>> min per suite x 3 suites], see [1]. Not sure yet what caused the increase,
>> but precommits used to be faster. Perhaps we have added a slow test or a
>> lot of new tests.
>> >> >>>>>> >> >> >>>>>>>>>> >
>> >> >>>>>> >> >> >>>>>>>>>> > [1]
>> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>> >> >>>>>> >> >> >>>>>>>>>> >
>> >> >>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
>> altay@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>> >>
>> >> >>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a
>> different suite sounds good. Anyone is interested in doing that?
>> >> >>>>>> >> >> >>>>>>>>>> >>
>> >> >>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn
>> Tymofieiev <va...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>> >>>
>> >> >>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
>> substantially, since precommit ITs are running sequentially with precommit
>> tests, unlike multiple precommit tests which run in parallel to each other.
>> >> >>>>>> >> >> >>>>>>>>>> >>>
>> >> >>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and
>> streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of
>> tests.
>> >> >>>>>> >> >> >>>>>>>>>> >>>
>> >> >>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet
>> Altay <al...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit.
>> Downside would be, when Chad tried to do something similar [1] it was noted
>> that the total time to run all precommit tests would increase and also
>> potentially increase the queue time.
>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a
>> smaller set of IT tests in precommits and run the whole suite as part of
>> post commit tests.
>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>> [1]
>> https://github.com/apache/beam/pull/9642
>> >> >>>>>> >> >> >>>>>>>>>> >>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
>> Tymofieiev <va...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to
>> Precommit IT tests into a separate suite from precommit tests, and run it
>> in parallel.
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian
>> Hulette <bh...@google.com> wrote:
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a
>> while now [1]. Just visually it looks like the average length is 1.5h or
>> so, but it spikes up to 2h. I've had several precommit runs get aborted due
>> to the 2 hour limit.
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up
>> above 1h back on 9/6 and the duration has been steadily rising since then.
>> Is there anything we can do about this?
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Brian
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >> >>>>>> >> >> >>>>>>>>>> >>>>>> [1]
>> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>>
>

Re: Python Precommit duration pushing 2 hours

Posted by Ning Kang <ni...@google.com>.
I'm removing the additional interactive test env + suite and add
[interactive] dependencies as extra dependencies in tests_require:
https://github.com/apache/beam/pull/10068

On Mon, Nov 11, 2019 at 2:15 PM Robert Bradshaw <ro...@google.com> wrote:

> On Fri, Nov 8, 2019 at 5:45 PM Ahmet Altay <al...@google.com> wrote:
> >
> > I looked at the log but I could not figure what is causing the timeout
> because the gradle scan links are missing. I sampled a few of the
> successful jobs, It seems like python 3.7 and python 2 are running 3 tests
> in serial {interactive, py37cython, py37gcp} and {docs, py27cython,
> py27gcp} respectively. These two versions are pushing the total time
> because other variants are now only running {cython, gcp} versions.
> >
> > I suggest breaking up docs, and interactive into 2 separate suites of
> their own. docs is actually faster than interactive,just separating that
> out to a new suite might help.
> >
> > Interactive was recently added (https://github.com/apache/beam/pull/9741).
> +Ning Kang could you separate interactive to new suite?
>
> I would ask why interactive is a separate tox configuration at all; I
> don't think there's a need to run every test again with a couple of
> extra dependencies (adding ~30 minutes to every presumbit). I think it
> would be much more valuable to run the (presumably relatively small)
> set of interactive tests in all modes.
>
> (The other suites are to guerentee the tests specifically run
> *without* installing gcp and *without* compiling with Cython.)
>
> > On Fri, Nov 8, 2019 at 11:09 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> Just saw another 2-hour timeout:
> >> https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
> >> perhaps we're not out of the woods yet (though in general things have
> >> been a lot better).
> >>
> >> On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
> >> >
> >> > GCP tests are already on separate locations. IO related tests are
> under /sdks/python/apache_beam/io/gcp and Dataflow related tests are under
> sdks/python/apache_beam/runners/dataflow. It should be a matter of changing
> gradle files to run either one of the base tests or GCP tests depending on
> the types of changes. I do not expect this to have any material impact on
> the precommit times because these two test suites take about exactly the
> same time to complete.
> >> >
> >> > #9985 is merged now. Precommit times on master branch dropped to ~1h
> 20 for the last 5 runs.
> >> >
> >> > On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com>
> wrote:
> >> >>
> >> >> +1 to moving the GCP tests outside of core. If there are issues that
> only show up on GCP tests but not in core, it might be an indication that
> there needs to be another test in core covering that, but I think that
> should be pretty rare.
> >> >>
> >> >> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org>
> wrote:
> >> >>>
> >> >>> +1 to moving forward with this
> >> >>>
> >> >>> Could we move GCP tests outside the core? Then only code changes
> touches/affecting GCP would cause them to run in precommit. Could still run
> them in postcommit in their own suite. If the core has reasonably stable
> abstractions that the connectors are built on, this should not change
> coverage much.
> >> >>>
> >> >>> Kenn
> >> >>>
> >> >>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com>
> wrote:
> >> >>>>
> >> >>>> PR for the proposed change:
> https://github.com/apache/beam/pull/9985
> >> >>>>
> >> >>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
> >> >>>>>
> >> >>>>> +1
> >> >>>>>
> >> >>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <
> robertwb@google.com> wrote:
> >> >>>>>>
> >> >>>>>> +1, this seems like a good step with a clear win.
> >> >>>>>>
> >> >>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com>
> wrote:
> >> >>>>>> >
> >> >>>>>> > Python precommits are still timing out on #9925. I am guessing
> that means this change would not be enough.
> >> >>>>>> >
> >> >>>>>> > I am proposing cutting down the number of test variants we run
> in precommits. Currently for each version we ran the following variants
> serially:
> >> >>>>>> > - base: Runs all unit tests with tox
> >> >>>>>> > - Cython: Installs cython and runs all unit tests as base
> version. The original purpose was to ensure that tests pass with or without
> cython. There is probably a huge overlap with base. (IIRC only a few coders
> have different slow vs fast tests.)
> >> >>>>>> > - GCP: Installs GCP dependencies and tests all base +
> additional gcp specific tests. The original purpose was to ensure that GCP
> is an optional component and all non-GCP tests still works without GCP
> components.
> >> >>>>>> >
> >> >>>>>> > We can reduce the list to cython + GCP tests only. This will
> cover the same group of tests and will check that tests pass with or
> without cython or GCP dependencies. This could reduce the precommit time by
> ~30 minutes.
> >> >>>>>> >
> >> >>>>>> > What do you think?
> >> >>>>>> >
> >> >>>>>> > Ahmet
> >> >>>>>> >
> >> >>>>>> >
> >> >>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <
> robertwb@google.com> wrote:
> >> >>>>>> >>
> >> >>>>>> >> https://github.com/apache/beam/pull/9925
> >> >>>>>> >>
> >> >>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com>
> wrote:
> >> >>>>>> >> >
> >> >>>>>> >> > I don't have the bandwidth right now to tackle this. Feel
> free to take it.
> >> >>>>>> >> >
> >> >>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
> robertwb@google.com> wrote:
> >> >>>>>> >> >>
> >> >>>>>> >> >> The Python SDK does as well. These calls are coming from
> >> >>>>>> >> >> to_runner_api, is_stateful_dofn, and
> validate_stateful_dofn which are
> >> >>>>>> >> >> invoked once per pipene or bundle. They are, however,
> surprisingly
> >> >>>>>> >> >> expensive. Even memoizing across those three calls should
> save a
> >> >>>>>> >> >> significant amount of time. Udi, did you want to tackle
> this?
> >> >>>>>> >> >>
> >> >>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being
> called 30
> >> >>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api
> being called
> >> >>>>>> >> >> 3111 times, so that in itself might be interesting to
> investigate.
> >> >>>>>> >> >>
> >> >>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <
> robert@frantil.com> wrote:
> >> >>>>>> >> >> >
> >> >>>>>> >> >> > As does the Go SDK. Invokers are memoized and when
> possible code is generated to avoid reflection.
> >> >>>>>> >> >> >
> >> >>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <
> klk@google.com> wrote:
> >> >>>>>> >> >> >>
> >> >>>>>> >> >> >> Noting for the benefit of the thread archive in case
> someone goes digging and wonders if this affects other SDKs: the Java SDK
> memoizes DoFnSignatures and generated DoFnInvoker classes.
> >> >>>>>> >> >> >>
> >> >>>>>> >> >> >> Kenn
> >> >>>>>> >> >> >>
> >> >>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <
> ehudm@google.com> wrote:
> >> >>>>>> >> >> >>>
> >> >>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
> >> >>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run
> pipelines.
> >> >>>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and
> now we're using less efficient ones.
> >> >>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down?
> (py2 only)
> >> >>>>>> >> >> >>>
> >> >>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg"
> on 2.7 and 3.7 and got these cool graphs (attached).
> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of
> CPU time (IIUC), gets called ~230k times
> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets
> called ~200k times
> >> >>>>>> >> >> >>>
> >> >>>>>> >> >> >>> After memoization of get_function_args_defaults:
> >> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
> >> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
> >> >>>>>> >> >> >>>
> >> >>>>>> >> >> >>>
> >> >>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >>>>>> >> >> >>>>
> >> >>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
> >> >>>>>> >> >> >>>>
> >> >>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >>>>>> >> >> >>>>>
> >> >>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and
> used a utility to track runtime for each test method. I found some of the
> following things:
> >> >>>>>> >> >> >>>>> - Total test methods run: 2665
> >> >>>>>> >> >> >>>>> - Total test runtime: 990 seconds
> >> >>>>>> >> >> >>>>> - Deciles of time spent:
> >> >>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
> >> >>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
> >> >>>>>> >> >> >>>>>   - 130 in the 18-27% range
> >> >>>>>> >> >> >>>>>   - 95 in the 27-36% range
> >> >>>>>> >> >> >>>>>   - 77
> >> >>>>>> >> >> >>>>>   - 66
> >> >>>>>> >> >> >>>>>   - 55
> >> >>>>>> >> >> >>>>>   - 46
> >> >>>>>> >> >> >>>>>   - 37
> >> >>>>>> >> >> >>>>>   - 24
> >> >>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This
> represents about 1 minute and a half.
> >> >>>>>> >> >> >>>>>
> >> >>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and
> get gradual improvements from there. Although it seems .. not dramatic ones
> : )
> >> >>>>>> >> >> >>>>>
> >> >>>>>> >> >> >>>>> FWIW I uploaded the results here:
> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
> >> >>>>>> >> >> >>>>>
> >> >>>>>> >> >> >>>>> The slowest 13 tests were:
> >> >>>>>> >> >> >>>>>
> >> >>>>>> >> >> >>>>>
> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
> >> >>>>>> >> >> >>>>>   5.253582000732422),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
> >> >>>>>> >> >> >>>>>   7.907713890075684),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
> >> >>>>>> >> >> >>>>>   5.237942934036255),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
> >> >>>>>> >> >> >>>>>   5.563946008682251),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
> >> >>>>>> >> >> >>>>>   5.680700063705444),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
> >> >>>>>> >> >> >>>>>   6.111238956451416),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
> >> >>>>>> >> >> >>>>>   6.007534980773926),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
> >> >>>>>> >> >> >>>>>   13.993916988372803),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
> >> >>>>>> >> >> >>>>>   6.3383049964904785),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
> >> >>>>>> >> >> >>>>>   9.157485008239746),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
> >> >>>>>> >> >> >>>>>   5.191173076629639),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
> >> >>>>>> >> >> >>>>>   6.2221620082855225),
> >> >>>>>> >> >> >>>>>
> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
> >> >>>>>> >> >> >>>>>   7.7187910079956055)]
> >> >>>>>> >> >> >>>>>
> >> >>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >>>>>> >> >> >>>>>>
> >> >>>>>> >> >> >>>>>> I have written
> https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest
> variations.
> >> >>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to
> start a discussion.
> >> >>>>>> >> >> >>>>>> I'll also try to figure out if there are other
> tests slowing down the suite significantly.
> >> >>>>>> >> >> >>>>>> Best
> >> >>>>>> >> >> >>>>>> -P.
> >> >>>>>> >> >> >>>>>>
> >> >>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev
> <va...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>
> >> >>>>>> >> >> >>>>>>> Thanks, Brian.
> >> >>>>>> >> >> >>>>>>> +Udi Meiri
> >> >>>>>> >> >> >>>>>>> As next step, it would be good to know whether
> slowdown is caused by tests in this PR, or its effect on other tests, and
> to confirm that only Python 2 codepaths were affected.
> >> >>>>>> >> >> >>>>>>>
> >> >>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
> bhulette@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>
> >> >>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
> and 9/15 to see if I could find the source of the spike that happened
> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
> search would reveal some mis-guided configuration change, but as far as I
> can tell 9283 just added a well-tested feature. I don't think there's
> anything to learn from that... I just wanted to circle back about it in
> case others are curious about that spike.
> >> >>>>>> >> >> >>>>>>>>
> >> >>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
> >> >>>>>> >> >> >>>>>>>>
> >> >>>>>> >> >> >>>>>>>> Brian
> >> >>>>>> >> >> >>>>>>>>
> >> >>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
> >> >>>>>> >> >> >>>>>>>>
> >> >>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>
> >> >>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the
> extra FnApiRunner configurations. Perhaps some of the multiworkers and some
> of the grpc versions?
> >> >>>>>> >> >> >>>>>>>>> Best
> >> >>>>>> >> >> >>>>>>>>> -P.
> >> >>>>>> >> >> >>>>>>>>>
> >> >>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw
> <ro...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>>
> >> >>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite
> expensive, taking 10-15+
> >> >>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test
> consists of a base class
> >> >>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and
> is then run in several
> >> >>>>>> >> >> >>>>>>>>>> configurations, many more of which (including
> some expensive ones)
> >> >>>>>> >> >> >>>>>>>>>> have been added lately.
> >> >>>>>> >> >> >>>>>>>>>>
> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
> >> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
> >> >>>>>> >> >> >>>>>>>>>>
> >> >>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
> permutations, or at
> >> >>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
> >> >>>>>> >> >> >>>>>>>>>>
> >> >>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn
> Tymofieiev
> >> >>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>> >
> >> >>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs
> are already running in parallel, albeit in the same suite. However it
> appears Python precommits became slower, especially Python 2 precommits [35
> min per suite x 3 suites], see [1]. Not sure yet what caused the increase,
> but precommits used to be faster. Perhaps we have added a slow test or a
> lot of new tests.
> >> >>>>>> >> >> >>>>>>>>>> >
> >> >>>>>> >> >> >>>>>>>>>> > [1]
> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
> >> >>>>>> >> >> >>>>>>>>>> >
> >> >>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
> altay@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>> >>
> >> >>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different
> suite sounds good. Anyone is interested in doing that?
> >> >>>>>> >> >> >>>>>>>>>> >>
> >> >>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn
> Tymofieiev <va...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>> >>>
> >> >>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
> substantially, since precommit ITs are running sequentially with precommit
> tests, unlike multiple precommit tests which run in parallel to each other.
> >> >>>>>> >> >> >>>>>>>>>> >>>
> >> >>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and
> streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of
> tests.
> >> >>>>>> >> >> >>>>>>>>>> >>>
> >> >>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay
> <al...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>> >>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit.
> Downside would be, when Chad tried to do something similar [1] it was noted
> that the total time to run all precommit tests would increase and also
> potentially increase the queue time.
> >> >>>>>> >> >> >>>>>>>>>> >>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a
> smaller set of IT tests in precommits and run the whole suite as part of
> post commit tests.
> >> >>>>>> >> >> >>>>>>>>>> >>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>> [1]
> https://github.com/apache/beam/pull/9642
> >> >>>>>> >> >> >>>>>>>>>> >>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
> Tymofieiev <va...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>> >>>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to
> Precommit IT tests into a separate suite from precommit tests, and run it
> in parallel.
> >> >>>>>> >> >> >>>>>>>>>> >>>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian
> Hulette <bh...@google.com> wrote:
> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a
> while now [1]. Just visually it looks like the average length is 1.5h or
> so, but it spikes up to 2h. I've had several precommit runs get aborted due
> to the 2 hour limit.
> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above
> 1h back on 9/6 and the duration has been steadily rising since then. Is
> there anything we can do about this?
> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>>>> Brian
> >> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >> >>>>>> >> >> >>>>>>>>>> >>>>>> [1]
> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>

Re: Python Precommit duration pushing 2 hours

Posted by Robert Bradshaw <ro...@google.com>.
On Fri, Nov 8, 2019 at 5:45 PM Ahmet Altay <al...@google.com> wrote:
>
> I looked at the log but I could not figure what is causing the timeout because the gradle scan links are missing. I sampled a few of the successful jobs, It seems like python 3.7 and python 2 are running 3 tests in serial {interactive, py37cython, py37gcp} and {docs, py27cython, py27gcp} respectively. These two versions are pushing the total time because other variants are now only running {cython, gcp} versions.
>
> I suggest breaking up docs, and interactive into 2 separate suites of their own. docs is actually faster than interactive,just separating that out to a new suite might help.
>
> Interactive was recently added (https://github.com/apache/beam/pull/9741). +Ning Kang could you separate interactive to new suite?

I would ask why interactive is a separate tox configuration at all; I
don't think there's a need to run every test again with a couple of
extra dependencies (adding ~30 minutes to every presumbit). I think it
would be much more valuable to run the (presumably relatively small)
set of interactive tests in all modes.

(The other suites are to guerentee the tests specifically run
*without* installing gcp and *without* compiling with Cython.)

> On Fri, Nov 8, 2019 at 11:09 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> Just saw another 2-hour timeout:
>> https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
>> perhaps we're not out of the woods yet (though in general things have
>> been a lot better).
>>
>> On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
>> >
>> > GCP tests are already on separate locations. IO related tests are under /sdks/python/apache_beam/io/gcp and Dataflow related tests are under sdks/python/apache_beam/runners/dataflow. It should be a matter of changing gradle files to run either one of the base tests or GCP tests depending on the types of changes. I do not expect this to have any material impact on the precommit times because these two test suites take about exactly the same time to complete.
>> >
>> > #9985 is merged now. Precommit times on master branch dropped to ~1h 20 for the last 5 runs.
>> >
>> > On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com> wrote:
>> >>
>> >> +1 to moving the GCP tests outside of core. If there are issues that only show up on GCP tests but not in core, it might be an indication that there needs to be another test in core covering that, but I think that should be pretty rare.
>> >>
>> >> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org> wrote:
>> >>>
>> >>> +1 to moving forward with this
>> >>>
>> >>> Could we move GCP tests outside the core? Then only code changes touches/affecting GCP would cause them to run in precommit. Could still run them in postcommit in their own suite. If the core has reasonably stable abstractions that the connectors are built on, this should not change coverage much.
>> >>>
>> >>> Kenn
>> >>>
>> >>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:
>> >>>>
>> >>>> PR for the proposed change: https://github.com/apache/beam/pull/9985
>> >>>>
>> >>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
>> >>>>>
>> >>>>> +1
>> >>>>>
>> >>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com> wrote:
>> >>>>>>
>> >>>>>> +1, this seems like a good step with a clear win.
>> >>>>>>
>> >>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>> >>>>>> >
>> >>>>>> > Python precommits are still timing out on #9925. I am guessing that means this change would not be enough.
>> >>>>>> >
>> >>>>>> > I am proposing cutting down the number of test variants we run in precommits. Currently for each version we ran the following variants serially:
>> >>>>>> > - base: Runs all unit tests with tox
>> >>>>>> > - Cython: Installs cython and runs all unit tests as base version. The original purpose was to ensure that tests pass with or without cython. There is probably a huge overlap with base. (IIRC only a few coders have different slow vs fast tests.)
>> >>>>>> > - GCP: Installs GCP dependencies and tests all base + additional gcp specific tests. The original purpose was to ensure that GCP is an optional component and all non-GCP tests still works without GCP components.
>> >>>>>> >
>> >>>>>> > We can reduce the list to cython + GCP tests only. This will cover the same group of tests and will check that tests pass with or without cython or GCP dependencies. This could reduce the precommit time by ~30 minutes.
>> >>>>>> >
>> >>>>>> > What do you think?
>> >>>>>> >
>> >>>>>> > Ahmet
>> >>>>>> >
>> >>>>>> >
>> >>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com> wrote:
>> >>>>>> >>
>> >>>>>> >> https://github.com/apache/beam/pull/9925
>> >>>>>> >>
>> >>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>> >>>>>> >> >
>> >>>>>> >> > I don't have the bandwidth right now to tackle this. Feel free to take it.
>> >>>>>> >> >
>> >>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com> wrote:
>> >>>>>> >> >>
>> >>>>>> >> >> The Python SDK does as well. These calls are coming from
>> >>>>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which are
>> >>>>>> >> >> invoked once per pipene or bundle. They are, however, surprisingly
>> >>>>>> >> >> expensive. Even memoizing across those three calls should save a
>> >>>>>> >> >> significant amount of time. Udi, did you want to tackle this?
>> >>>>>> >> >>
>> >>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
>> >>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
>> >>>>>> >> >> 3111 times, so that in itself might be interesting to investigate.
>> >>>>>> >> >>
>> >>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com> wrote:
>> >>>>>> >> >> >
>> >>>>>> >> >> > As does the Go SDK. Invokers are memoized and when possible code is generated to avoid reflection.
>> >>>>>> >> >> >
>> >>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com> wrote:
>> >>>>>> >> >> >>
>> >>>>>> >> >> >> Noting for the benefit of the thread archive in case someone goes digging and wonders if this affects other SDKs: the Java SDK memoizes DoFnSignatures and generated DoFnInvoker classes.
>> >>>>>> >> >> >>
>> >>>>>> >> >> >> Kenn
>> >>>>>> >> >> >>
>> >>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com> wrote:
>> >>>>>> >> >> >>>
>> >>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>> >>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
>> >>>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and now we're using less efficient ones.
>> >>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>> >>>>>> >> >> >>>
>> >>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and 3.7 and got these cool graphs (attached).
>> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time (IIUC), gets called ~230k times
>> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k times
>> >>>>>> >> >> >>>
>> >>>>>> >> >> >>> After memoization of get_function_args_defaults:
>> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>> >>>>>> >> >> >>>
>> >>>>>> >> >> >>>
>> >>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <pa...@google.com> wrote:
>> >>>>>> >> >> >>>>
>> >>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>> >>>>>> >> >> >>>>
>> >>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <pa...@google.com> wrote:
>> >>>>>> >> >> >>>>>
>> >>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a utility to track runtime for each test method. I found some of the following things:
>> >>>>>> >> >> >>>>> - Total test methods run: 2665
>> >>>>>> >> >> >>>>> - Total test runtime: 990 seconds
>> >>>>>> >> >> >>>>> - Deciles of time spent:
>> >>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>> >>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>> >>>>>> >> >> >>>>>   - 130 in the 18-27% range
>> >>>>>> >> >> >>>>>   - 95 in the 27-36% range
>> >>>>>> >> >> >>>>>   - 77
>> >>>>>> >> >> >>>>>   - 66
>> >>>>>> >> >> >>>>>   - 55
>> >>>>>> >> >> >>>>>   - 46
>> >>>>>> >> >> >>>>>   - 37
>> >>>>>> >> >> >>>>>   - 24
>> >>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents about 1 minute and a half.
>> >>>>>> >> >> >>>>>
>> >>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and get gradual improvements from there. Although it seems .. not dramatic ones : )
>> >>>>>> >> >> >>>>>
>> >>>>>> >> >> >>>>> FWIW I uploaded the results here: https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>> >>>>>> >> >> >>>>>
>> >>>>>> >> >> >>>>> The slowest 13 tests were:
>> >>>>>> >> >> >>>>>
>> >>>>>> >> >> >>>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>> >>>>>> >> >> >>>>>   5.253582000732422),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>> >>>>>> >> >> >>>>>   7.907713890075684),
>> >>>>>> >> >> >>>>>  ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>> >>>>>> >> >> >>>>>   5.237942934036255),
>> >>>>>> >> >> >>>>>  ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>> >>>>>> >> >> >>>>>   5.563946008682251),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>> >>>>>> >> >> >>>>>   5.680700063705444),
>> >>>>>> >> >> >>>>>  ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>> >>>>>> >> >> >>>>>   6.111238956451416),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>> >>>>>> >> >> >>>>>   6.007534980773926),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>> >>>>>> >> >> >>>>>   13.993916988372803),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>> >>>>>> >> >> >>>>>   6.3383049964904785),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>> >>>>>> >> >> >>>>>   9.157485008239746),
>> >>>>>> >> >> >>>>>  ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>> >>>>>> >> >> >>>>>   5.191173076629639),
>> >>>>>> >> >> >>>>>  ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>> >>>>>> >> >> >>>>>   6.2221620082855225),
>> >>>>>> >> >> >>>>>  ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>> >>>>>> >> >> >>>>>   7.7187910079956055)]
>> >>>>>> >> >> >>>>>
>> >>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <pa...@google.com> wrote:
>> >>>>>> >> >> >>>>>>
>> >>>>>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest variations.
>> >>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a discussion.
>> >>>>>> >> >> >>>>>> I'll also try to figure out if there are other tests slowing down the suite significantly.
>> >>>>>> >> >> >>>>>> Best
>> >>>>>> >> >> >>>>>> -P.
>> >>>>>> >> >> >>>>>>
>> >>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>
>> >>>>>> >> >> >>>>>>> Thanks, Brian.
>> >>>>>> >> >> >>>>>>> +Udi Meiri
>> >>>>>> >> >> >>>>>>> As next step, it would be good to know whether slowdown is caused by tests in this PR, or its effect on other tests, and to confirm that only Python 2 codepaths were affected.
>> >>>>>> >> >> >>>>>>>
>> >>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <bh...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>
>> >>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1 and 9/15 to see if I could find the source of the spike that happened around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this search would reveal some mis-guided configuration change, but as far as I can tell 9283 just added a well-tested feature. I don't think there's anything to learn from that... I just wanted to circle back about it in case others are curious about that spike.
>> >>>>>> >> >> >>>>>>>>
>> >>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>> >>>>>> >> >> >>>>>>>>
>> >>>>>> >> >> >>>>>>>> Brian
>> >>>>>> >> >> >>>>>>>>
>> >>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>> >>>>>> >> >> >>>>>>>>
>> >>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <pa...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>
>> >>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra FnApiRunner configurations. Perhaps some of the multiworkers and some of the grpc versions?
>> >>>>>> >> >> >>>>>>>>> Best
>> >>>>>> >> >> >>>>>>>>> -P.
>> >>>>>> >> >> >>>>>>>>>
>> >>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <ro...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>>
>> >>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive, taking 10-15+
>> >>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists of a base class
>> >>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is then run in several
>> >>>>>> >> >> >>>>>>>>>> configurations, many more of which (including some expensive ones)
>> >>>>>> >> >> >>>>>>>>>> have been added lately.
>> >>>>>> >> >> >>>>>>>>>>
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>> >>>>>> >> >> >>>>>>>>>>
>> >>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these permutations, or at
>> >>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>> >>>>>> >> >> >>>>>>>>>>
>> >>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>> >>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>> >
>> >>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are already running in parallel, albeit in the same suite. However it appears Python precommits became slower, especially Python 2 precommits [35 min per suite x 3 suites], see [1]. Not sure yet what caused the increase, but precommits used to be faster. Perhaps we have added a slow test or a lot of new tests.
>> >>>>>> >> >> >>>>>>>>>> >
>> >>>>>> >> >> >>>>>>>>>> > [1] https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>> >>>>>> >> >> >>>>>>>>>> >
>> >>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <al...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>> >>
>> >>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite sounds good. Anyone is interested in doing that?
>> >>>>>> >> >> >>>>>>>>>> >>
>> >>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>> >>>
>> >>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time substantially, since precommit ITs are running sequentially with precommit tests, unlike multiple precommit tests which run in parallel to each other.
>> >>>>>> >> >> >>>>>>>>>> >>>
>> >>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>> >>>>>> >> >> >>>>>>>>>> >>>
>> >>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <al...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>> >>>>
>> >>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would be, when Chad tried to do something similar [1] it was noted that the total time to run all precommit tests would increase and also potentially increase the queue time.
>> >>>>>> >> >> >>>>>>>>>> >>>>
>> >>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of IT tests in precommits and run the whole suite as part of post commit tests.
>> >>>>>> >> >> >>>>>>>>>> >>>>
>> >>>>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>> >>>>>> >> >> >>>>>>>>>> >>>>
>> >>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn Tymofieiev <va...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>> >>>>>
>> >>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT tests into a separate suite from precommit tests, and run it in parallel.
>> >>>>>> >> >> >>>>>>>>>> >>>>>
>> >>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <bh...@google.com> wrote:
>> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now [1]. Just visually it looks like the average length is 1.5h or so, but it spikes up to 2h. I've had several precommit runs get aborted due to the 2 hour limit.
>> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back on 9/6 and the duration has been steadily rising since then. Is there anything we can do about this?
>> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >>>>>> >> >> >>>>>>>>>> >>>>>> Brian
>> >>>>>> >> >> >>>>>>>>>> >>>>>>
>> >>>>>> >> >> >>>>>>>>>> >>>>>> [1] http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4

Re: Python Precommit duration pushing 2 hours

Posted by Ahmet Altay <al...@google.com>.
I looked at the log but I could not figure what is causing the timeout
because the gradle scan links are missing. I sampled a few of the
successful jobs, It seems like python 3.7 and python 2 are running 3 tests
in serial {interactive, py37cython, py37gcp} and {docs, py27cython,
py27gcp} respectively. These two versions are pushing the total time
because other variants are now only running {cython, gcp} versions.

I suggest breaking up docs, and interactive into 2 separate suites of their
own. docs is actually faster than interactive,just separating that out to a
new suite might help.

Interactive was recently added
(https://github.com/apache/beam/pull/9741). +Ning
Kang <ni...@google.com> could you separate interactive to new suite?

Ahmet

On Fri, Nov 8, 2019 at 11:09 AM Robert Bradshaw <ro...@google.com> wrote:

> Just saw another 2-hour timeout:
> https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
> perhaps we're not out of the woods yet (though in general things have
> been a lot better).
>
> On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
> >
> > GCP tests are already on separate locations. IO related tests are under
> /sdks/python/apache_beam/io/gcp and Dataflow related tests are under
> sdks/python/apache_beam/runners/dataflow. It should be a matter of changing
> gradle files to run either one of the base tests or GCP tests depending on
> the types of changes. I do not expect this to have any material impact on
> the precommit times because these two test suites take about exactly the
> same time to complete.
> >
> > #9985 is merged now. Precommit times on master branch dropped to ~1h 20
> for the last 5 runs.
> >
> > On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com>
> wrote:
> >>
> >> +1 to moving the GCP tests outside of core. If there are issues that
> only show up on GCP tests but not in core, it might be an indication that
> there needs to be another test in core covering that, but I think that
> should be pretty rare.
> >>
> >> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org> wrote:
> >>>
> >>> +1 to moving forward with this
> >>>
> >>> Could we move GCP tests outside the core? Then only code changes
> touches/affecting GCP would cause them to run in precommit. Could still run
> them in postcommit in their own suite. If the core has reasonably stable
> abstractions that the connectors are built on, this should not change
> coverage much.
> >>>
> >>> Kenn
> >>>
> >>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:
> >>>>
> >>>> PR for the proposed change: https://github.com/apache/beam/pull/9985
> >>>>
> >>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
> >>>>>
> >>>>> +1
> >>>>>
> >>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com>
> wrote:
> >>>>>>
> >>>>>> +1, this seems like a good step with a clear win.
> >>>>>>
> >>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com>
> wrote:
> >>>>>> >
> >>>>>> > Python precommits are still timing out on #9925. I am guessing
> that means this change would not be enough.
> >>>>>> >
> >>>>>> > I am proposing cutting down the number of test variants we run in
> precommits. Currently for each version we ran the following variants
> serially:
> >>>>>> > - base: Runs all unit tests with tox
> >>>>>> > - Cython: Installs cython and runs all unit tests as base
> version. The original purpose was to ensure that tests pass with or without
> cython. There is probably a huge overlap with base. (IIRC only a few coders
> have different slow vs fast tests.)
> >>>>>> > - GCP: Installs GCP dependencies and tests all base + additional
> gcp specific tests. The original purpose was to ensure that GCP is an
> optional component and all non-GCP tests still works without GCP components.
> >>>>>> >
> >>>>>> > We can reduce the list to cython + GCP tests only. This will
> cover the same group of tests and will check that tests pass with or
> without cython or GCP dependencies. This could reduce the precommit time by
> ~30 minutes.
> >>>>>> >
> >>>>>> > What do you think?
> >>>>>> >
> >>>>>> > Ahmet
> >>>>>> >
> >>>>>> >
> >>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <
> robertwb@google.com> wrote:
> >>>>>> >>
> >>>>>> >> https://github.com/apache/beam/pull/9925
> >>>>>> >>
> >>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com>
> wrote:
> >>>>>> >> >
> >>>>>> >> > I don't have the bandwidth right now to tackle this. Feel free
> to take it.
> >>>>>> >> >
> >>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
> robertwb@google.com> wrote:
> >>>>>> >> >>
> >>>>>> >> >> The Python SDK does as well. These calls are coming from
> >>>>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn
> which are
> >>>>>> >> >> invoked once per pipene or bundle. They are, however,
> surprisingly
> >>>>>> >> >> expensive. Even memoizing across those three calls should
> save a
> >>>>>> >> >> significant amount of time. Udi, did you want to tackle this?
> >>>>>> >> >>
> >>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being
> called 30
> >>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being
> called
> >>>>>> >> >> 3111 times, so that in itself might be interesting to
> investigate.
> >>>>>> >> >>
> >>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <
> robert@frantil.com> wrote:
> >>>>>> >> >> >
> >>>>>> >> >> > As does the Go SDK. Invokers are memoized and when possible
> code is generated to avoid reflection.
> >>>>>> >> >> >
> >>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <
> klk@google.com> wrote:
> >>>>>> >> >> >>
> >>>>>> >> >> >> Noting for the benefit of the thread archive in case
> someone goes digging and wonders if this affects other SDKs: the Java SDK
> memoizes DoFnSignatures and generated DoFnInvoker classes.
> >>>>>> >> >> >>
> >>>>>> >> >> >> Kenn
> >>>>>> >> >> >>
> >>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <
> ehudm@google.com> wrote:
> >>>>>> >> >> >>>
> >>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
> >>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run
> pipelines.
> >>>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and
> now we're using less efficient ones.
> >>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2
> only)
> >>>>>> >> >> >>>
> >>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on
> 2.7 and 3.7 and got these cool graphs (attached).
> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU
> time (IIUC), gets called ~230k times
> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called
> ~200k times
> >>>>>> >> >> >>>
> >>>>>> >> >> >>> After memoization of get_function_args_defaults:
> >>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
> >>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
> >>>>>> >> >> >>>
> >>>>>> >> >> >>>
> >>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >>>>>> >> >> >>>>
> >>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
> >>>>>> >> >> >>>>
> >>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >>>>>> >> >> >>>>>
> >>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and
> used a utility to track runtime for each test method. I found some of the
> following things:
> >>>>>> >> >> >>>>> - Total test methods run: 2665
> >>>>>> >> >> >>>>> - Total test runtime: 990 seconds
> >>>>>> >> >> >>>>> - Deciles of time spent:
> >>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
> >>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
> >>>>>> >> >> >>>>>   - 130 in the 18-27% range
> >>>>>> >> >> >>>>>   - 95 in the 27-36% range
> >>>>>> >> >> >>>>>   - 77
> >>>>>> >> >> >>>>>   - 66
> >>>>>> >> >> >>>>>   - 55
> >>>>>> >> >> >>>>>   - 46
> >>>>>> >> >> >>>>>   - 37
> >>>>>> >> >> >>>>>   - 24
> >>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This
> represents about 1 minute and a half.
> >>>>>> >> >> >>>>>
> >>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and get
> gradual improvements from there. Although it seems .. not dramatic ones : )
> >>>>>> >> >> >>>>>
> >>>>>> >> >> >>>>> FWIW I uploaded the results here:
> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
> >>>>>> >> >> >>>>>
> >>>>>> >> >> >>>>> The slowest 13 tests were:
> >>>>>> >> >> >>>>>
> >>>>>> >> >> >>>>>
> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
> >>>>>> >> >> >>>>>   5.253582000732422),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
> >>>>>> >> >> >>>>>   7.907713890075684),
> >>>>>> >> >> >>>>>
> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
> >>>>>> >> >> >>>>>   5.237942934036255),
> >>>>>> >> >> >>>>>
> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
> >>>>>> >> >> >>>>>   5.563946008682251),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
> >>>>>> >> >> >>>>>   5.680700063705444),
> >>>>>> >> >> >>>>>
> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
> >>>>>> >> >> >>>>>   6.111238956451416),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
> >>>>>> >> >> >>>>>   6.007534980773926),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
> >>>>>> >> >> >>>>>   13.993916988372803),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
> >>>>>> >> >> >>>>>   6.3383049964904785),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
> >>>>>> >> >> >>>>>   9.157485008239746),
> >>>>>> >> >> >>>>>
> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
> >>>>>> >> >> >>>>>   5.191173076629639),
> >>>>>> >> >> >>>>>
> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
> >>>>>> >> >> >>>>>   6.2221620082855225),
> >>>>>> >> >> >>>>>
> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
> >>>>>> >> >> >>>>>   7.7187910079956055)]
> >>>>>> >> >> >>>>>
> >>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >>>>>> >> >> >>>>>>
> >>>>>> >> >> >>>>>> I have written
> https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest
> variations.
> >>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start
> a discussion.
> >>>>>> >> >> >>>>>> I'll also try to figure out if there are other tests
> slowing down the suite significantly.
> >>>>>> >> >> >>>>>> Best
> >>>>>> >> >> >>>>>> -P.
> >>>>>> >> >> >>>>>>
> >>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
> valentyn@google.com> wrote:
> >>>>>> >> >> >>>>>>>
> >>>>>> >> >> >>>>>>> Thanks, Brian.
> >>>>>> >> >> >>>>>>> +Udi Meiri
> >>>>>> >> >> >>>>>>> As next step, it would be good to know whether
> slowdown is caused by tests in this PR, or its effect on other tests, and
> to confirm that only Python 2 codepaths were affected.
> >>>>>> >> >> >>>>>>>
> >>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
> bhulette@google.com> wrote:
> >>>>>> >> >> >>>>>>>>
> >>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
> and 9/15 to see if I could find the source of the spike that happened
> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
> search would reveal some mis-guided configuration change, but as far as I
> can tell 9283 just added a well-tested feature. I don't think there's
> anything to learn from that... I just wanted to circle back about it in
> case others are curious about that spike.
> >>>>>> >> >> >>>>>>>>
> >>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
> >>>>>> >> >> >>>>>>>>
> >>>>>> >> >> >>>>>>>> Brian
> >>>>>> >> >> >>>>>>>>
> >>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
> >>>>>> >> >> >>>>>>>>
> >>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>
> >>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra
> FnApiRunner configurations. Perhaps some of the multiworkers and some of
> the grpc versions?
> >>>>>> >> >> >>>>>>>>> Best
> >>>>>> >> >> >>>>>>>>> -P.
> >>>>>> >> >> >>>>>>>>>
> >>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
> robertwb@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>>
> >>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite
> expensive, taking 10-15+
> >>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test
> consists of a base class
> >>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is
> then run in several
> >>>>>> >> >> >>>>>>>>>> configurations, many more of which (including some
> expensive ones)
> >>>>>> >> >> >>>>>>>>>> have been added lately.
> >>>>>> >> >> >>>>>>>>>>
> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
> >>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
> >>>>>> >> >> >>>>>>>>>>
> >>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
> permutations, or at
> >>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
> >>>>>> >> >> >>>>>>>>>>
> >>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn
> Tymofieiev
> >>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>> >
> >>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs
> are already running in parallel, albeit in the same suite. However it
> appears Python precommits became slower, especially Python 2 precommits [35
> min per suite x 3 suites], see [1]. Not sure yet what caused the increase,
> but precommits used to be faster. Perhaps we have added a slow test or a
> lot of new tests.
> >>>>>> >> >> >>>>>>>>>> >
> >>>>>> >> >> >>>>>>>>>> > [1]
> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
> >>>>>> >> >> >>>>>>>>>> >
> >>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
> altay@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>> >>
> >>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different
> suite sounds good. Anyone is interested in doing that?
> >>>>>> >> >> >>>>>>>>>> >>
> >>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn
> Tymofieiev <va...@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>> >>>
> >>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
> substantially, since precommit ITs are running sequentially with precommit
> tests, unlike multiple precommit tests which run in parallel to each other.
> >>>>>> >> >> >>>>>>>>>> >>>
> >>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and
> streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of
> tests.
> >>>>>> >> >> >>>>>>>>>> >>>
> >>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
> altay@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>> >>>>
> >>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside
> would be, when Chad tried to do something similar [1] it was noted that the
> total time to run all precommit tests would increase and also potentially
> increase the queue time.
> >>>>>> >> >> >>>>>>>>>> >>>>
> >>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller
> set of IT tests in precommits and run the whole suite as part of post
> commit tests.
> >>>>>> >> >> >>>>>>>>>> >>>>
> >>>>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
> >>>>>> >> >> >>>>>>>>>> >>>>
> >>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
> Tymofieiev <va...@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>> >>>>>
> >>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit
> IT tests into a separate suite from precommit tests, and run it in parallel.
> >>>>>> >> >> >>>>>>>>>> >>>>>
> >>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian
> Hulette <bh...@google.com> wrote:
> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while
> now [1]. Just visually it looks like the average length is 1.5h or so, but
> it spikes up to 2h. I've had several precommit runs get aborted due to the
> 2 hour limit.
> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h
> back on 9/6 and the duration has been steadily rising since then. Is there
> anything we can do about this?
> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >>>>>> >> >> >>>>>>>>>> >>>>>> Brian
> >>>>>> >> >> >>>>>>>>>> >>>>>>
> >>>>>> >> >> >>>>>>>>>> >>>>>> [1]
> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>

Re: Python Precommit duration pushing 2 hours

Posted by Robert Bradshaw <ro...@google.com>.
Just saw another 2-hour timeout:
https://builds.apache.org/job/beam_PreCommit_Python_Commit/9440/ , so
perhaps we're not out of the woods yet (though in general things have
been a lot better).

On Tue, Nov 5, 2019 at 10:52 AM Ahmet Altay <al...@google.com> wrote:
>
> GCP tests are already on separate locations. IO related tests are under /sdks/python/apache_beam/io/gcp and Dataflow related tests are under sdks/python/apache_beam/runners/dataflow. It should be a matter of changing gradle files to run either one of the base tests or GCP tests depending on the types of changes. I do not expect this to have any material impact on the precommit times because these two test suites take about exactly the same time to complete.
>
> #9985 is merged now. Precommit times on master branch dropped to ~1h 20 for the last 5 runs.
>
> On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com> wrote:
>>
>> +1 to moving the GCP tests outside of core. If there are issues that only show up on GCP tests but not in core, it might be an indication that there needs to be another test in core covering that, but I think that should be pretty rare.
>>
>> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org> wrote:
>>>
>>> +1 to moving forward with this
>>>
>>> Could we move GCP tests outside the core? Then only code changes touches/affecting GCP would cause them to run in precommit. Could still run them in postcommit in their own suite. If the core has reasonably stable abstractions that the connectors are built on, this should not change coverage much.
>>>
>>> Kenn
>>>
>>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:
>>>>
>>>> PR for the proposed change: https://github.com/apache/beam/pull/9985
>>>>
>>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
>>>>>
>>>>> +1
>>>>>
>>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com> wrote:
>>>>>>
>>>>>> +1, this seems like a good step with a clear win.
>>>>>>
>>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>>>>>> >
>>>>>> > Python precommits are still timing out on #9925. I am guessing that means this change would not be enough.
>>>>>> >
>>>>>> > I am proposing cutting down the number of test variants we run in precommits. Currently for each version we ran the following variants serially:
>>>>>> > - base: Runs all unit tests with tox
>>>>>> > - Cython: Installs cython and runs all unit tests as base version. The original purpose was to ensure that tests pass with or without cython. There is probably a huge overlap with base. (IIRC only a few coders have different slow vs fast tests.)
>>>>>> > - GCP: Installs GCP dependencies and tests all base + additional gcp specific tests. The original purpose was to ensure that GCP is an optional component and all non-GCP tests still works without GCP components.
>>>>>> >
>>>>>> > We can reduce the list to cython + GCP tests only. This will cover the same group of tests and will check that tests pass with or without cython or GCP dependencies. This could reduce the precommit time by ~30 minutes.
>>>>>> >
>>>>>> > What do you think?
>>>>>> >
>>>>>> > Ahmet
>>>>>> >
>>>>>> >
>>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com> wrote:
>>>>>> >>
>>>>>> >> https://github.com/apache/beam/pull/9925
>>>>>> >>
>>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>>>>>> >> >
>>>>>> >> > I don't have the bandwidth right now to tackle this. Feel free to take it.
>>>>>> >> >
>>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com> wrote:
>>>>>> >> >>
>>>>>> >> >> The Python SDK does as well. These calls are coming from
>>>>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which are
>>>>>> >> >> invoked once per pipene or bundle. They are, however, surprisingly
>>>>>> >> >> expensive. Even memoizing across those three calls should save a
>>>>>> >> >> significant amount of time. Udi, did you want to tackle this?
>>>>>> >> >>
>>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
>>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
>>>>>> >> >> 3111 times, so that in itself might be interesting to investigate.
>>>>>> >> >>
>>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com> wrote:
>>>>>> >> >> >
>>>>>> >> >> > As does the Go SDK. Invokers are memoized and when possible code is generated to avoid reflection.
>>>>>> >> >> >
>>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com> wrote:
>>>>>> >> >> >>
>>>>>> >> >> >> Noting for the benefit of the thread archive in case someone goes digging and wonders if this affects other SDKs: the Java SDK memoizes DoFnSignatures and generated DoFnInvoker classes.
>>>>>> >> >> >>
>>>>>> >> >> >> Kenn
>>>>>> >> >> >>
>>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com> wrote:
>>>>>> >> >> >>>
>>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
>>>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and now we're using less efficient ones.
>>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>>>>>> >> >> >>>
>>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and 3.7 and got these cool graphs (attached).
>>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time (IIUC), gets called ~230k times
>>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k times
>>>>>> >> >> >>>
>>>>>> >> >> >>> After memoization of get_function_args_defaults:
>>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>>>>>> >> >> >>>
>>>>>> >> >> >>>
>>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <pa...@google.com> wrote:
>>>>>> >> >> >>>>
>>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>>>>>> >> >> >>>>
>>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <pa...@google.com> wrote:
>>>>>> >> >> >>>>>
>>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a utility to track runtime for each test method. I found some of the following things:
>>>>>> >> >> >>>>> - Total test methods run: 2665
>>>>>> >> >> >>>>> - Total test runtime: 990 seconds
>>>>>> >> >> >>>>> - Deciles of time spent:
>>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>>>>>> >> >> >>>>>   - 130 in the 18-27% range
>>>>>> >> >> >>>>>   - 95 in the 27-36% range
>>>>>> >> >> >>>>>   - 77
>>>>>> >> >> >>>>>   - 66
>>>>>> >> >> >>>>>   - 55
>>>>>> >> >> >>>>>   - 46
>>>>>> >> >> >>>>>   - 37
>>>>>> >> >> >>>>>   - 24
>>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents about 1 minute and a half.
>>>>>> >> >> >>>>>
>>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and get gradual improvements from there. Although it seems .. not dramatic ones : )
>>>>>> >> >> >>>>>
>>>>>> >> >> >>>>> FWIW I uploaded the results here: https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>>>>>> >> >> >>>>>
>>>>>> >> >> >>>>> The slowest 13 tests were:
>>>>>> >> >> >>>>>
>>>>>> >> >> >>>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>>>>>> >> >> >>>>>   5.253582000732422),
>>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>>>>>> >> >> >>>>>   7.907713890075684),
>>>>>> >> >> >>>>>  ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>>>>>> >> >> >>>>>   5.237942934036255),
>>>>>> >> >> >>>>>  ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>>>>>> >> >> >>>>>   5.563946008682251),
>>>>>> >> >> >>>>>  ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>>>>>> >> >> >>>>>   5.680700063705444),
>>>>>> >> >> >>>>>  ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>>>>>> >> >> >>>>>   6.111238956451416),
>>>>>> >> >> >>>>>  ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>>>>>> >> >> >>>>>   6.007534980773926),
>>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>>>>>> >> >> >>>>>   13.993916988372803),
>>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>>>>>> >> >> >>>>>   6.3383049964904785),
>>>>>> >> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>>>>>> >> >> >>>>>   9.157485008239746),
>>>>>> >> >> >>>>>  ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>>>>>> >> >> >>>>>   5.191173076629639),
>>>>>> >> >> >>>>>  ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>>>>>> >> >> >>>>>   6.2221620082855225),
>>>>>> >> >> >>>>>  ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>>>>>> >> >> >>>>>   7.7187910079956055)]
>>>>>> >> >> >>>>>
>>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <pa...@google.com> wrote:
>>>>>> >> >> >>>>>>
>>>>>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest variations.
>>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a discussion.
>>>>>> >> >> >>>>>> I'll also try to figure out if there are other tests slowing down the suite significantly.
>>>>>> >> >> >>>>>> Best
>>>>>> >> >> >>>>>> -P.
>>>>>> >> >> >>>>>>
>>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>>>>>> >> >> >>>>>>>
>>>>>> >> >> >>>>>>> Thanks, Brian.
>>>>>> >> >> >>>>>>> +Udi Meiri
>>>>>> >> >> >>>>>>> As next step, it would be good to know whether slowdown is caused by tests in this PR, or its effect on other tests, and to confirm that only Python 2 codepaths were affected.
>>>>>> >> >> >>>>>>>
>>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <bh...@google.com> wrote:
>>>>>> >> >> >>>>>>>>
>>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1 and 9/15 to see if I could find the source of the spike that happened around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this search would reveal some mis-guided configuration change, but as far as I can tell 9283 just added a well-tested feature. I don't think there's anything to learn from that... I just wanted to circle back about it in case others are curious about that spike.
>>>>>> >> >> >>>>>>>>
>>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>>>>>> >> >> >>>>>>>>
>>>>>> >> >> >>>>>>>> Brian
>>>>>> >> >> >>>>>>>>
>>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>>>>>> >> >> >>>>>>>>
>>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <pa...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>
>>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra FnApiRunner configurations. Perhaps some of the multiworkers and some of the grpc versions?
>>>>>> >> >> >>>>>>>>> Best
>>>>>> >> >> >>>>>>>>> -P.
>>>>>> >> >> >>>>>>>>>
>>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <ro...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>>
>>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive, taking 10-15+
>>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists of a base class
>>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is then run in several
>>>>>> >> >> >>>>>>>>>> configurations, many more of which (including some expensive ones)
>>>>>> >> >> >>>>>>>>>> have been added lately.
>>>>>> >> >> >>>>>>>>>>
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>>>>>> >> >> >>>>>>>>>>
>>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these permutations, or at
>>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>>>>>> >> >> >>>>>>>>>>
>>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>> >
>>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are already running in parallel, albeit in the same suite. However it appears Python precommits became slower, especially Python 2 precommits [35 min per suite x 3 suites], see [1]. Not sure yet what caused the increase, but precommits used to be faster. Perhaps we have added a slow test or a lot of new tests.
>>>>>> >> >> >>>>>>>>>> >
>>>>>> >> >> >>>>>>>>>> > [1] https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>>>>>> >> >> >>>>>>>>>> >
>>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <al...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>> >>
>>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite sounds good. Anyone is interested in doing that?
>>>>>> >> >> >>>>>>>>>> >>
>>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>> >>>
>>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time substantially, since precommit ITs are running sequentially with precommit tests, unlike multiple precommit tests which run in parallel to each other.
>>>>>> >> >> >>>>>>>>>> >>>
>>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>>>>>> >> >> >>>>>>>>>> >>>
>>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <al...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>> >>>>
>>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would be, when Chad tried to do something similar [1] it was noted that the total time to run all precommit tests would increase and also potentially increase the queue time.
>>>>>> >> >> >>>>>>>>>> >>>>
>>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of IT tests in precommits and run the whole suite as part of post commit tests.
>>>>>> >> >> >>>>>>>>>> >>>>
>>>>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>>>>>> >> >> >>>>>>>>>> >>>>
>>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn Tymofieiev <va...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>> >>>>>
>>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT tests into a separate suite from precommit tests, and run it in parallel.
>>>>>> >> >> >>>>>>>>>> >>>>>
>>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <bh...@google.com> wrote:
>>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now [1]. Just visually it looks like the average length is 1.5h or so, but it spikes up to 2h. I've had several precommit runs get aborted due to the 2 hour limit.
>>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back on 9/6 and the duration has been steadily rising since then. Is there anything we can do about this?
>>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>>> >> >> >>>>>>>>>> >>>>>> Brian
>>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>>> >> >> >>>>>>>>>> >>>>>> [1] http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4

Re: Python Precommit duration pushing 2 hours

Posted by Ahmet Altay <al...@google.com>.
GCP tests are already on separate locations. IO related tests are
under /sdks/python/apache_beam/io/gcp and Dataflow related tests are
under sdks/python/apache_beam/runners/dataflow. It should be a matter of
changing gradle files to run either one of the base tests or GCP tests
depending on the types of changes. I do not expect this to have any
material impact on the precommit times because these two test suites take
about exactly the same time to complete.

#9985 is merged now. Precommit times on master branch dropped to ~1h 20 for
the last 5 runs.

On Tue, Nov 5, 2019 at 10:12 AM David Cavazos <dc...@google.com> wrote:

> +1 to moving the GCP tests outside of core. If there are issues that only
> show up on GCP tests but not in core, it might be an indication that there
> needs to be another test in core covering that, but I think that should be
> pretty rare.
>
> On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org> wrote:
>
>> +1 to moving forward with this
>>
>> Could we move GCP tests outside the core? Then only code changes
>> touches/affecting GCP would cause them to run in precommit. Could still run
>> them in postcommit in their own suite. If the core has reasonably stable
>> abstractions that the connectors are built on, this should not change
>> coverage much.
>>
>> Kenn
>>
>> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:
>>
>>> PR for the proposed change: https://github.com/apache/beam/pull/9985
>>>
>>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
>>>
>>>> +1
>>>>
>>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>>
>>>>> +1, this seems like a good step with a clear win.
>>>>>
>>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>>>>> >
>>>>> > Python precommits are still timing out on #9925. I am guessing that
>>>>> means this change would not be enough.
>>>>> >
>>>>> > I am proposing cutting down the number of test variants we run in
>>>>> precommits. Currently for each version we ran the following variants
>>>>> serially:
>>>>> > - base: Runs all unit tests with tox
>>>>> > - Cython: Installs cython and runs all unit tests as base version.
>>>>> The original purpose was to ensure that tests pass with or without cython.
>>>>> There is probably a huge overlap with base. (IIRC only a few coders have
>>>>> different slow vs fast tests.)
>>>>> > - GCP: Installs GCP dependencies and tests all base + additional gcp
>>>>> specific tests. The original purpose was to ensure that GCP is an optional
>>>>> component and all non-GCP tests still works without GCP components.
>>>>> >
>>>>> > We can reduce the list to cython + GCP tests only. This will cover
>>>>> the same group of tests and will check that tests pass with or without
>>>>> cython or GCP dependencies. This could reduce the precommit time by ~30
>>>>> minutes.
>>>>> >
>>>>> > What do you think?
>>>>> >
>>>>> > Ahmet
>>>>> >
>>>>> >
>>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >>
>>>>> >> https://github.com/apache/beam/pull/9925
>>>>> >>
>>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com>
>>>>> wrote:
>>>>> >> >
>>>>> >> > I don't have the bandwidth right now to tackle this. Feel free to
>>>>> take it.
>>>>> >> >
>>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >> >>
>>>>> >> >> The Python SDK does as well. These calls are coming from
>>>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn
>>>>> which are
>>>>> >> >> invoked once per pipene or bundle. They are, however,
>>>>> surprisingly
>>>>> >> >> expensive. Even memoizing across those three calls should save a
>>>>> >> >> significant amount of time. Udi, did you want to tackle this?
>>>>> >> >>
>>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called
>>>>> 30
>>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being
>>>>> called
>>>>> >> >> 3111 times, so that in itself might be interesting to
>>>>> investigate.
>>>>> >> >>
>>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com>
>>>>> wrote:
>>>>> >> >> >
>>>>> >> >> > As does the Go SDK. Invokers are memoized and when possible
>>>>> code is generated to avoid reflection.
>>>>> >> >> >
>>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com>
>>>>> wrote:
>>>>> >> >> >>
>>>>> >> >> >> Noting for the benefit of the thread archive in case someone
>>>>> goes digging and wonders if this affects other SDKs: the Java SDK memoizes
>>>>> DoFnSignatures and generated DoFnInvoker classes.
>>>>> >> >> >>
>>>>> >> >> >> Kenn
>>>>> >> >> >>
>>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com>
>>>>> wrote:
>>>>> >> >> >>>
>>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>>>>> >> >> >>> 1. I added a lot of test cases, some with locally run
>>>>> pipelines.
>>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and now
>>>>> we're using less efficient ones.
>>>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>>>>> >> >> >>>
>>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7
>>>>> and 3.7 and got these cool graphs (attached).
>>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU
>>>>> time (IIUC), gets called ~230k times
>>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called
>>>>> ~200k times
>>>>> >> >> >>>
>>>>> >> >> >>> After memoization of get_function_args_defaults:
>>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>>>>> >> >> >>>
>>>>> >> >> >>>
>>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
>>>>> pabloem@google.com> wrote:
>>>>> >> >> >>>>
>>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>>>>> >> >> >>>>
>>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
>>>>> pabloem@google.com> wrote:
>>>>> >> >> >>>>>
>>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used
>>>>> a utility to track runtime for each test method. I found some of the
>>>>> following things:
>>>>> >> >> >>>>> - Total test methods run: 2665
>>>>> >> >> >>>>> - Total test runtime: 990 seconds
>>>>> >> >> >>>>> - Deciles of time spent:
>>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>>>>> >> >> >>>>>   - 130 in the 18-27% range
>>>>> >> >> >>>>>   - 95 in the 27-36% range
>>>>> >> >> >>>>>   - 77
>>>>> >> >> >>>>>   - 66
>>>>> >> >> >>>>>   - 55
>>>>> >> >> >>>>>   - 46
>>>>> >> >> >>>>>   - 37
>>>>> >> >> >>>>>   - 24
>>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents
>>>>> about 1 minute and a half.
>>>>> >> >> >>>>>
>>>>> >> >> >>>>> We may be able to look at the slowest X tests, and get
>>>>> gradual improvements from there. Although it seems .. not dramatic ones : )
>>>>> >> >> >>>>>
>>>>> >> >> >>>>> FWIW I uploaded the results here:
>>>>> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>>>>> >> >> >>>>>
>>>>> >> >> >>>>> The slowest 13 tests were:
>>>>> >> >> >>>>>
>>>>> >> >> >>>>>
>>>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>>>>> >> >> >>>>>   5.253582000732422),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>>>>> >> >> >>>>>   7.907713890075684),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>>>>> >> >> >>>>>   5.237942934036255),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>>>>> >> >> >>>>>   5.563946008682251),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>>>>> >> >> >>>>>   5.680700063705444),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>>>>> >> >> >>>>>   6.111238956451416),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>>>>> >> >> >>>>>   6.007534980773926),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>>>>> >> >> >>>>>   13.993916988372803),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>>>>> >> >> >>>>>   6.3383049964904785),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>>>>> >> >> >>>>>   9.157485008239746),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>>>>> >> >> >>>>>   5.191173076629639),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>>>>> >> >> >>>>>   6.2221620082855225),
>>>>> >> >> >>>>>
>>>>> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>>>>> >> >> >>>>>   7.7187910079956055)]
>>>>> >> >> >>>>>
>>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
>>>>> pabloem@google.com> wrote:
>>>>> >> >> >>>>>>
>>>>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910
>>>>> to reduce FnApiRunnerTest variations.
>>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a
>>>>> discussion.
>>>>> >> >> >>>>>> I'll also try to figure out if there are other tests
>>>>> slowing down the suite significantly.
>>>>> >> >> >>>>>> Best
>>>>> >> >> >>>>>> -P.
>>>>> >> >> >>>>>>
>>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
>>>>> valentyn@google.com> wrote:
>>>>> >> >> >>>>>>>
>>>>> >> >> >>>>>>> Thanks, Brian.
>>>>> >> >> >>>>>>> +Udi Meiri
>>>>> >> >> >>>>>>> As next step, it would be good to know whether slowdown
>>>>> is caused by tests in this PR, or its effect on other tests, and to confirm
>>>>> that only Python 2 codepaths were affected.
>>>>> >> >> >>>>>>>
>>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
>>>>> bhulette@google.com> wrote:
>>>>> >> >> >>>>>>>>
>>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
>>>>> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
>>>>> and 9/15 to see if I could find the source of the spike that happened
>>>>> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
>>>>> search would reveal some mis-guided configuration change, but as far as I
>>>>> can tell 9283 just added a well-tested feature. I don't think there's
>>>>> anything to learn from that... I just wanted to circle back about it in
>>>>> case others are curious about that spike.
>>>>> >> >> >>>>>>>>
>>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>>>>> >> >> >>>>>>>>
>>>>> >> >> >>>>>>>> Brian
>>>>> >> >> >>>>>>>>
>>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>>>>> >> >> >>>>>>>>
>>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
>>>>> pabloem@google.com> wrote:
>>>>> >> >> >>>>>>>>>
>>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra
>>>>> FnApiRunner configurations. Perhaps some of the multiworkers and some of
>>>>> the grpc versions?
>>>>> >> >> >>>>>>>>> Best
>>>>> >> >> >>>>>>>>> -P.
>>>>> >> >> >>>>>>>>>
>>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
>>>>> robertwb@google.com> wrote:
>>>>> >> >> >>>>>>>>>>
>>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite
>>>>> expensive, taking 10-15+
>>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists
>>>>> of a base class
>>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is
>>>>> then run in several
>>>>> >> >> >>>>>>>>>> configurations, many more of which (including some
>>>>> expensive ones)
>>>>> >> >> >>>>>>>>>> have been added lately.
>>>>> >> >> >>>>>>>>>>
>>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>> class
>>>>> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>> class
>>>>> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>> class
>>>>> FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>> class
>>>>> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>> class
>>>>> FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>> class
>>>>> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>>>>> >> >> >>>>>>>>>>
>>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
>>>>> permutations, or at
>>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>>>>> >> >> >>>>>>>>>>
>>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>>>>> >> >> >>>>>>>>>> >
>>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are
>>>>> already running in parallel, albeit in the same suite. However it appears
>>>>> Python precommits became slower, especially Python 2 precommits [35 min per
>>>>> suite x 3 suites], see [1]. Not sure yet what caused the increase, but
>>>>> precommits used to be faster. Perhaps we have added a slow test or a lot of
>>>>> new tests.
>>>>> >> >> >>>>>>>>>> >
>>>>> >> >> >>>>>>>>>> > [1]
>>>>> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>>>>> >> >> >>>>>>>>>> >
>>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
>>>>> altay@google.com> wrote:
>>>>> >> >> >>>>>>>>>> >>
>>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite
>>>>> sounds good. Anyone is interested in doing that?
>>>>> >> >> >>>>>>>>>> >>
>>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn
>>>>> Tymofieiev <va...@google.com> wrote:
>>>>> >> >> >>>>>>>>>> >>>
>>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
>>>>> substantially, since precommit ITs are running sequentially with precommit
>>>>> tests, unlike multiple precommit tests which run in parallel to each other.
>>>>> >> >> >>>>>>>>>> >>>
>>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming
>>>>> wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>>>>> >> >> >>>>>>>>>> >>>
>>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
>>>>> altay@google.com> wrote:
>>>>> >> >> >>>>>>>>>> >>>>
>>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside
>>>>> would be, when Chad tried to do something similar [1] it was noted that the
>>>>> total time to run all precommit tests would increase and also potentially
>>>>> increase the queue time.
>>>>> >> >> >>>>>>>>>> >>>>
>>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set
>>>>> of IT tests in precommits and run the whole suite as part of post commit
>>>>> tests.
>>>>> >> >> >>>>>>>>>> >>>>
>>>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>>>>> >> >> >>>>>>>>>> >>>>
>>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
>>>>> Tymofieiev <va...@google.com> wrote:
>>>>> >> >> >>>>>>>>>> >>>>>
>>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT
>>>>> tests into a separate suite from precommit tests, and run it in parallel.
>>>>> >> >> >>>>>>>>>> >>>>>
>>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <
>>>>> bhulette@google.com> wrote:
>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now
>>>>> [1]. Just visually it looks like the average length is 1.5h or so, but it
>>>>> spikes up to 2h. I've had several precommit runs get aborted due to the 2
>>>>> hour limit.
>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h
>>>>> back on 9/6 and the duration has been steadily rising since then. Is there
>>>>> anything we can do about this?
>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>> >> >> >>>>>>>>>> >>>>>> Brian
>>>>> >> >> >>>>>>>>>> >>>>>>
>>>>> >> >> >>>>>>>>>> >>>>>> [1]
>>>>> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>>>>>
>>>>

Re: Python Precommit duration pushing 2 hours

Posted by David Cavazos <dc...@google.com>.
+1 to moving the GCP tests outside of core. If there are issues that only
show up on GCP tests but not in core, it might be an indication that there
needs to be another test in core covering that, but I think that should be
pretty rare.

On Mon, Nov 4, 2019 at 8:33 PM Kenneth Knowles <ke...@apache.org> wrote:

> +1 to moving forward with this
>
> Could we move GCP tests outside the core? Then only code changes
> touches/affecting GCP would cause them to run in precommit. Could still run
> them in postcommit in their own suite. If the core has reasonably stable
> abstractions that the connectors are built on, this should not change
> coverage much.
>
> Kenn
>
> On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:
>
>> PR for the proposed change: https://github.com/apache/beam/pull/9985
>>
>> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
>>
>>> +1
>>>
>>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>>
>>>> +1, this seems like a good step with a clear win.
>>>>
>>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>>>> >
>>>> > Python precommits are still timing out on #9925. I am guessing that
>>>> means this change would not be enough.
>>>> >
>>>> > I am proposing cutting down the number of test variants we run in
>>>> precommits. Currently for each version we ran the following variants
>>>> serially:
>>>> > - base: Runs all unit tests with tox
>>>> > - Cython: Installs cython and runs all unit tests as base version.
>>>> The original purpose was to ensure that tests pass with or without cython.
>>>> There is probably a huge overlap with base. (IIRC only a few coders have
>>>> different slow vs fast tests.)
>>>> > - GCP: Installs GCP dependencies and tests all base + additional gcp
>>>> specific tests. The original purpose was to ensure that GCP is an optional
>>>> component and all non-GCP tests still works without GCP components.
>>>> >
>>>> > We can reduce the list to cython + GCP tests only. This will cover
>>>> the same group of tests and will check that tests pass with or without
>>>> cython or GCP dependencies. This could reduce the precommit time by ~30
>>>> minutes.
>>>> >
>>>> > What do you think?
>>>> >
>>>> > Ahmet
>>>> >
>>>> >
>>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com>
>>>> wrote:
>>>> >>
>>>> >> https://github.com/apache/beam/pull/9925
>>>> >>
>>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>>>> >> >
>>>> >> > I don't have the bandwidth right now to tackle this. Feel free to
>>>> take it.
>>>> >> >
>>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
>>>> robertwb@google.com> wrote:
>>>> >> >>
>>>> >> >> The Python SDK does as well. These calls are coming from
>>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which
>>>> are
>>>> >> >> invoked once per pipene or bundle. They are, however, surprisingly
>>>> >> >> expensive. Even memoizing across those three calls should save a
>>>> >> >> significant amount of time. Udi, did you want to tackle this?
>>>> >> >>
>>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called
>>>> 30
>>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
>>>> >> >> 3111 times, so that in itself might be interesting to investigate.
>>>> >> >>
>>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com>
>>>> wrote:
>>>> >> >> >
>>>> >> >> > As does the Go SDK. Invokers are memoized and when possible
>>>> code is generated to avoid reflection.
>>>> >> >> >
>>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com>
>>>> wrote:
>>>> >> >> >>
>>>> >> >> >> Noting for the benefit of the thread archive in case someone
>>>> goes digging and wonders if this affects other SDKs: the Java SDK memoizes
>>>> DoFnSignatures and generated DoFnInvoker classes.
>>>> >> >> >>
>>>> >> >> >> Kenn
>>>> >> >> >>
>>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com>
>>>> wrote:
>>>> >> >> >>>
>>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>>>> >> >> >>> 1. I added a lot of test cases, some with locally run
>>>> pipelines.
>>>> >> >> >>> 2. The PR somehow changed how coders are selected, and now
>>>> we're using less efficient ones.
>>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>>>> >> >> >>>
>>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7
>>>> and 3.7 and got these cool graphs (attached).
>>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time
>>>> (IIUC), gets called ~230k times
>>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called
>>>> ~200k times
>>>> >> >> >>>
>>>> >> >> >>> After memoization of get_function_args_defaults:
>>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>>>> >> >> >>>
>>>> >> >> >>>
>>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
>>>> pabloem@google.com> wrote:
>>>> >> >> >>>>
>>>> >> >> >>>> *not deciles, but 9-percentiles : )
>>>> >> >> >>>>
>>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
>>>> pabloem@google.com> wrote:
>>>> >> >> >>>>>
>>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a
>>>> utility to track runtime for each test method. I found some of the
>>>> following things:
>>>> >> >> >>>>> - Total test methods run: 2665
>>>> >> >> >>>>> - Total test runtime: 990 seconds
>>>> >> >> >>>>> - Deciles of time spent:
>>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>>>> >> >> >>>>>   - 130 in the 18-27% range
>>>> >> >> >>>>>   - 95 in the 27-36% range
>>>> >> >> >>>>>   - 77
>>>> >> >> >>>>>   - 66
>>>> >> >> >>>>>   - 55
>>>> >> >> >>>>>   - 46
>>>> >> >> >>>>>   - 37
>>>> >> >> >>>>>   - 24
>>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents
>>>> about 1 minute and a half.
>>>> >> >> >>>>>
>>>> >> >> >>>>> We may be able to look at the slowest X tests, and get
>>>> gradual improvements from there. Although it seems .. not dramatic ones : )
>>>> >> >> >>>>>
>>>> >> >> >>>>> FWIW I uploaded the results here:
>>>> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>>>> >> >> >>>>>
>>>> >> >> >>>>> The slowest 13 tests were:
>>>> >> >> >>>>>
>>>> >> >> >>>>>
>>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>>>> >> >> >>>>>   5.253582000732422),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>>>> >> >> >>>>>   7.907713890075684),
>>>> >> >> >>>>>
>>>> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>>>> >> >> >>>>>   5.237942934036255),
>>>> >> >> >>>>>
>>>> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>>>> >> >> >>>>>   5.563946008682251),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>>>> >> >> >>>>>   5.680700063705444),
>>>> >> >> >>>>>
>>>> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>>>> >> >> >>>>>   6.111238956451416),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>>>> >> >> >>>>>   6.007534980773926),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>>>> >> >> >>>>>   13.993916988372803),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>>>> >> >> >>>>>   6.3383049964904785),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>>>> >> >> >>>>>   9.157485008239746),
>>>> >> >> >>>>>
>>>> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>>>> >> >> >>>>>   5.191173076629639),
>>>> >> >> >>>>>
>>>> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>>>> >> >> >>>>>   6.2221620082855225),
>>>> >> >> >>>>>
>>>> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>>>> >> >> >>>>>   7.7187910079956055)]
>>>> >> >> >>>>>
>>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
>>>> pabloem@google.com> wrote:
>>>> >> >> >>>>>>
>>>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910
>>>> to reduce FnApiRunnerTest variations.
>>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a
>>>> discussion.
>>>> >> >> >>>>>> I'll also try to figure out if there are other tests
>>>> slowing down the suite significantly.
>>>> >> >> >>>>>> Best
>>>> >> >> >>>>>> -P.
>>>> >> >> >>>>>>
>>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
>>>> valentyn@google.com> wrote:
>>>> >> >> >>>>>>>
>>>> >> >> >>>>>>> Thanks, Brian.
>>>> >> >> >>>>>>> +Udi Meiri
>>>> >> >> >>>>>>> As next step, it would be good to know whether slowdown
>>>> is caused by tests in this PR, or its effect on other tests, and to confirm
>>>> that only Python 2 codepaths were affected.
>>>> >> >> >>>>>>>
>>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
>>>> bhulette@google.com> wrote:
>>>> >> >> >>>>>>>>
>>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
>>>> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
>>>> and 9/15 to see if I could find the source of the spike that happened
>>>> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
>>>> search would reveal some mis-guided configuration change, but as far as I
>>>> can tell 9283 just added a well-tested feature. I don't think there's
>>>> anything to learn from that... I just wanted to circle back about it in
>>>> case others are curious about that spike.
>>>> >> >> >>>>>>>>
>>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>>>> >> >> >>>>>>>>
>>>> >> >> >>>>>>>> Brian
>>>> >> >> >>>>>>>>
>>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>>>> >> >> >>>>>>>>
>>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
>>>> pabloem@google.com> wrote:
>>>> >> >> >>>>>>>>>
>>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra
>>>> FnApiRunner configurations. Perhaps some of the multiworkers and some of
>>>> the grpc versions?
>>>> >> >> >>>>>>>>> Best
>>>> >> >> >>>>>>>>> -P.
>>>> >> >> >>>>>>>>>
>>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
>>>> robertwb@google.com> wrote:
>>>> >> >> >>>>>>>>>>
>>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite
>>>> expensive, taking 10-15+
>>>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists
>>>> of a base class
>>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is
>>>> then run in several
>>>> >> >> >>>>>>>>>> configurations, many more of which (including some
>>>> expensive ones)
>>>> >> >> >>>>>>>>>> have been added lately.
>>>> >> >> >>>>>>>>>>
>>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>> class
>>>> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>> class
>>>> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>> class
>>>> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>> class
>>>> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>>>> >> >> >>>>>>>>>>
>>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
>>>> permutations, or at
>>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>>>> >> >> >>>>>>>>>>
>>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>>>> >> >> >>>>>>>>>> >
>>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are
>>>> already running in parallel, albeit in the same suite. However it appears
>>>> Python precommits became slower, especially Python 2 precommits [35 min per
>>>> suite x 3 suites], see [1]. Not sure yet what caused the increase, but
>>>> precommits used to be faster. Perhaps we have added a slow test or a lot of
>>>> new tests.
>>>> >> >> >>>>>>>>>> >
>>>> >> >> >>>>>>>>>> > [1]
>>>> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>>>> >> >> >>>>>>>>>> >
>>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
>>>> altay@google.com> wrote:
>>>> >> >> >>>>>>>>>> >>
>>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite
>>>> sounds good. Anyone is interested in doing that?
>>>> >> >> >>>>>>>>>> >>
>>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev
>>>> <va...@google.com> wrote:
>>>> >> >> >>>>>>>>>> >>>
>>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
>>>> substantially, since precommit ITs are running sequentially with precommit
>>>> tests, unlike multiple precommit tests which run in parallel to each other.
>>>> >> >> >>>>>>>>>> >>>
>>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming
>>>> wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>>>> >> >> >>>>>>>>>> >>>
>>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
>>>> altay@google.com> wrote:
>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside
>>>> would be, when Chad tried to do something similar [1] it was noted that the
>>>> total time to run all precommit tests would increase and also potentially
>>>> increase the queue time.
>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set
>>>> of IT tests in precommits and run the whole suite as part of post commit
>>>> tests.
>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>>>> >> >> >>>>>>>>>> >>>>
>>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
>>>> Tymofieiev <va...@google.com> wrote:
>>>> >> >> >>>>>>>>>> >>>>>
>>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT
>>>> tests into a separate suite from precommit tests, and run it in parallel.
>>>> >> >> >>>>>>>>>> >>>>>
>>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <
>>>> bhulette@google.com> wrote:
>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now
>>>> [1]. Just visually it looks like the average length is 1.5h or so, but it
>>>> spikes up to 2h. I've had several precommit runs get aborted due to the 2
>>>> hour limit.
>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h
>>>> back on 9/6 and the duration has been steadily rising since then. Is there
>>>> anything we can do about this?
>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >> >>>>>>>>>> >>>>>> Brian
>>>> >> >> >>>>>>>>>> >>>>>>
>>>> >> >> >>>>>>>>>> >>>>>> [1]
>>>> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>>>>
>>>

Re: Python Precommit duration pushing 2 hours

Posted by Kenneth Knowles <ke...@apache.org>.
+1 to moving forward with this

Could we move GCP tests outside the core? Then only code changes
touches/affecting GCP would cause them to run in precommit. Could still run
them in postcommit in their own suite. If the core has reasonably stable
abstractions that the connectors are built on, this should not change
coverage much.

Kenn

On Mon, Nov 4, 2019 at 1:55 PM Ahmet Altay <al...@google.com> wrote:

> PR for the proposed change: https://github.com/apache/beam/pull/9985
>
> On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:
>
>> +1
>>
>> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com>
>> wrote:
>>
>>> +1, this seems like a good step with a clear win.
>>>
>>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>>> >
>>> > Python precommits are still timing out on #9925. I am guessing that
>>> means this change would not be enough.
>>> >
>>> > I am proposing cutting down the number of test variants we run in
>>> precommits. Currently for each version we ran the following variants
>>> serially:
>>> > - base: Runs all unit tests with tox
>>> > - Cython: Installs cython and runs all unit tests as base version. The
>>> original purpose was to ensure that tests pass with or without cython.
>>> There is probably a huge overlap with base. (IIRC only a few coders have
>>> different slow vs fast tests.)
>>> > - GCP: Installs GCP dependencies and tests all base + additional gcp
>>> specific tests. The original purpose was to ensure that GCP is an optional
>>> component and all non-GCP tests still works without GCP components.
>>> >
>>> > We can reduce the list to cython + GCP tests only. This will cover the
>>> same group of tests and will check that tests pass with or without cython
>>> or GCP dependencies. This could reduce the precommit time by ~30 minutes.
>>> >
>>> > What do you think?
>>> >
>>> > Ahmet
>>> >
>>> >
>>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com>
>>> wrote:
>>> >>
>>> >> https://github.com/apache/beam/pull/9925
>>> >>
>>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>>> >> >
>>> >> > I don't have the bandwidth right now to tackle this. Feel free to
>>> take it.
>>> >> >
>>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >>
>>> >> >> The Python SDK does as well. These calls are coming from
>>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which
>>> are
>>> >> >> invoked once per pipene or bundle. They are, however, surprisingly
>>> >> >> expensive. Even memoizing across those three calls should save a
>>> >> >> significant amount of time. Udi, did you want to tackle this?
>>> >> >>
>>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
>>> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
>>> >> >> 3111 times, so that in itself might be interesting to investigate.
>>> >> >>
>>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com>
>>> wrote:
>>> >> >> >
>>> >> >> > As does the Go SDK. Invokers are memoized and when possible code
>>> is generated to avoid reflection.
>>> >> >> >
>>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com>
>>> wrote:
>>> >> >> >>
>>> >> >> >> Noting for the benefit of the thread archive in case someone
>>> goes digging and wonders if this affects other SDKs: the Java SDK memoizes
>>> DoFnSignatures and generated DoFnInvoker classes.
>>> >> >> >>
>>> >> >> >> Kenn
>>> >> >> >>
>>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com>
>>> wrote:
>>> >> >> >>>
>>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>>> >> >> >>> 1. I added a lot of test cases, some with locally run
>>> pipelines.
>>> >> >> >>> 2. The PR somehow changed how coders are selected, and now
>>> we're using less efficient ones.
>>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>>> >> >> >>>
>>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7
>>> and 3.7 and got these cool graphs (attached).
>>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time
>>> (IIUC), gets called ~230k times
>>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k
>>> times
>>> >> >> >>>
>>> >> >> >>> After memoization of get_function_args_defaults:
>>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>>> >> >> >>>
>>> >> >> >>>
>>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >> >>>>
>>> >> >> >>>> *not deciles, but 9-percentiles : )
>>> >> >> >>>>
>>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >> >>>>>
>>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a
>>> utility to track runtime for each test method. I found some of the
>>> following things:
>>> >> >> >>>>> - Total test methods run: 2665
>>> >> >> >>>>> - Total test runtime: 990 seconds
>>> >> >> >>>>> - Deciles of time spent:
>>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>>> >> >> >>>>>   - 173 in the 9-18% rang3e
>>> >> >> >>>>>   - 130 in the 18-27% range
>>> >> >> >>>>>   - 95 in the 27-36% range
>>> >> >> >>>>>   - 77
>>> >> >> >>>>>   - 66
>>> >> >> >>>>>   - 55
>>> >> >> >>>>>   - 46
>>> >> >> >>>>>   - 37
>>> >> >> >>>>>   - 24
>>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents
>>> about 1 minute and a half.
>>> >> >> >>>>>
>>> >> >> >>>>> We may be able to look at the slowest X tests, and get
>>> gradual improvements from there. Although it seems .. not dramatic ones : )
>>> >> >> >>>>>
>>> >> >> >>>>> FWIW I uploaded the results here:
>>> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>>> >> >> >>>>>
>>> >> >> >>>>> The slowest 13 tests were:
>>> >> >> >>>>>
>>> >> >> >>>>>
>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>>> >> >> >>>>>   5.253582000732422),
>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>>> >> >> >>>>>   7.907713890075684),
>>> >> >> >>>>>
>>> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>>> >> >> >>>>>   5.237942934036255),
>>> >> >> >>>>>
>>> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>>> >> >> >>>>>   5.563946008682251),
>>> >> >> >>>>>
>>> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>>> >> >> >>>>>   5.680700063705444),
>>> >> >> >>>>>
>>> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>>> >> >> >>>>>   6.111238956451416),
>>> >> >> >>>>>
>>> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>>> >> >> >>>>>   6.007534980773926),
>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>>> >> >> >>>>>   13.993916988372803),
>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>>> >> >> >>>>>   6.3383049964904785),
>>> >> >> >>>>>
>>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>>> >> >> >>>>>   9.157485008239746),
>>> >> >> >>>>>
>>> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>>> >> >> >>>>>   5.191173076629639),
>>> >> >> >>>>>
>>> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>>> >> >> >>>>>   6.2221620082855225),
>>> >> >> >>>>>
>>> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>>> >> >> >>>>>   7.7187910079956055)]
>>> >> >> >>>>>
>>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >> >>>>>>
>>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to
>>> reduce FnApiRunnerTest variations.
>>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a
>>> discussion.
>>> >> >> >>>>>> I'll also try to figure out if there are other tests
>>> slowing down the suite significantly.
>>> >> >> >>>>>> Best
>>> >> >> >>>>>> -P.
>>> >> >> >>>>>>
>>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
>>> valentyn@google.com> wrote:
>>> >> >> >>>>>>>
>>> >> >> >>>>>>> Thanks, Brian.
>>> >> >> >>>>>>> +Udi Meiri
>>> >> >> >>>>>>> As next step, it would be good to know whether slowdown is
>>> caused by tests in this PR, or its effect on other tests, and to confirm
>>> that only Python 2 codepaths were affected.
>>> >> >> >>>>>>>
>>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
>>> bhulette@google.com> wrote:
>>> >> >> >>>>>>>>
>>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
>>> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
>>> and 9/15 to see if I could find the source of the spike that happened
>>> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
>>> search would reveal some mis-guided configuration change, but as far as I
>>> can tell 9283 just added a well-tested feature. I don't think there's
>>> anything to learn from that... I just wanted to circle back about it in
>>> case others are curious about that spike.
>>> >> >> >>>>>>>>
>>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>>> >> >> >>>>>>>>
>>> >> >> >>>>>>>> Brian
>>> >> >> >>>>>>>>
>>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>>> >> >> >>>>>>>>
>>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
>>> pabloem@google.com> wrote:
>>> >> >> >>>>>>>>>
>>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra
>>> FnApiRunner configurations. Perhaps some of the multiworkers and some of
>>> the grpc versions?
>>> >> >> >>>>>>>>> Best
>>> >> >> >>>>>>>>> -P.
>>> >> >> >>>>>>>>>
>>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
>>> robertwb@google.com> wrote:
>>> >> >> >>>>>>>>>>
>>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive,
>>> taking 10-15+
>>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists
>>> of a base class
>>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is then
>>> run in several
>>> >> >> >>>>>>>>>> configurations, many more of which (including some
>>> expensive ones)
>>> >> >> >>>>>>>>>> have been added lately.
>>> >> >> >>>>>>>>>>
>>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>>> >> >> >>>>>>>>>> class
>>> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>>> >> >> >>>>>>>>>>
>>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
>>> permutations, or at
>>> >> >> >>>>>>>>>> least not all tests in all permutations.
>>> >> >> >>>>>>>>>>
>>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>>> >> >> >>>>>>>>>> >
>>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are
>>> already running in parallel, albeit in the same suite. However it appears
>>> Python precommits became slower, especially Python 2 precommits [35 min per
>>> suite x 3 suites], see [1]. Not sure yet what caused the increase, but
>>> precommits used to be faster. Perhaps we have added a slow test or a lot of
>>> new tests.
>>> >> >> >>>>>>>>>> >
>>> >> >> >>>>>>>>>> > [1]
>>> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>>> >> >> >>>>>>>>>> >
>>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
>>> altay@google.com> wrote:
>>> >> >> >>>>>>>>>> >>
>>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite
>>> sounds good. Anyone is interested in doing that?
>>> >> >> >>>>>>>>>> >>
>>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <
>>> valentyn@google.com> wrote:
>>> >> >> >>>>>>>>>> >>>
>>> >> >> >>>>>>>>>> >>> This should not increase the queue time
>>> substantially, since precommit ITs are running sequentially with precommit
>>> tests, unlike multiple precommit tests which run in parallel to each other.
>>> >> >> >>>>>>>>>> >>>
>>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming
>>> wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>>> >> >> >>>>>>>>>> >>>
>>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
>>> altay@google.com> wrote:
>>> >> >> >>>>>>>>>> >>>>
>>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside
>>> would be, when Chad tried to do something similar [1] it was noted that the
>>> total time to run all precommit tests would increase and also potentially
>>> increase the queue time.
>>> >> >> >>>>>>>>>> >>>>
>>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of
>>> IT tests in precommits and run the whole suite as part of post commit tests.
>>> >> >> >>>>>>>>>> >>>>
>>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>>> >> >> >>>>>>>>>> >>>>
>>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
>>> Tymofieiev <va...@google.com> wrote:
>>> >> >> >>>>>>>>>> >>>>>
>>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT
>>> tests into a separate suite from precommit tests, and run it in parallel.
>>> >> >> >>>>>>>>>> >>>>>
>>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <
>>> bhulette@google.com> wrote:
>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now
>>> [1]. Just visually it looks like the average length is 1.5h or so, but it
>>> spikes up to 2h. I've had several precommit runs get aborted due to the 2
>>> hour limit.
>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back
>>> on 9/6 and the duration has been steadily rising since then. Is there
>>> anything we can do about this?
>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >> >>>>>>>>>> >>>>>> Brian
>>> >> >> >>>>>>>>>> >>>>>>
>>> >> >> >>>>>>>>>> >>>>>> [1]
>>> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>>>
>>

Re: Python Precommit duration pushing 2 hours

Posted by Ahmet Altay <al...@google.com>.
PR for the proposed change: https://github.com/apache/beam/pull/9985

On Mon, Nov 4, 2019 at 1:35 PM Udi Meiri <eh...@google.com> wrote:

> +1
>
> On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com>
> wrote:
>
>> +1, this seems like a good step with a clear win.
>>
>> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>> >
>> > Python precommits are still timing out on #9925. I am guessing that
>> means this change would not be enough.
>> >
>> > I am proposing cutting down the number of test variants we run in
>> precommits. Currently for each version we ran the following variants
>> serially:
>> > - base: Runs all unit tests with tox
>> > - Cython: Installs cython and runs all unit tests as base version. The
>> original purpose was to ensure that tests pass with or without cython.
>> There is probably a huge overlap with base. (IIRC only a few coders have
>> different slow vs fast tests.)
>> > - GCP: Installs GCP dependencies and tests all base + additional gcp
>> specific tests. The original purpose was to ensure that GCP is an optional
>> component and all non-GCP tests still works without GCP components.
>> >
>> > We can reduce the list to cython + GCP tests only. This will cover the
>> same group of tests and will check that tests pass with or without cython
>> or GCP dependencies. This could reduce the precommit time by ~30 minutes.
>> >
>> > What do you think?
>> >
>> > Ahmet
>> >
>> >
>> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com>
>> wrote:
>> >>
>> >> https://github.com/apache/beam/pull/9925
>> >>
>> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>> >> >
>> >> > I don't have the bandwidth right now to tackle this. Feel free to
>> take it.
>> >> >
>> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <
>> robertwb@google.com> wrote:
>> >> >>
>> >> >> The Python SDK does as well. These calls are coming from
>> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which
>> are
>> >> >> invoked once per pipene or bundle. They are, however, surprisingly
>> >> >> expensive. Even memoizing across those three calls should save a
>> >> >> significant amount of time. Udi, did you want to tackle this?
>> >> >>
>> >> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
>> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
>> >> >> 3111 times, so that in itself might be interesting to investigate.
>> >> >>
>> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com>
>> wrote:
>> >> >> >
>> >> >> > As does the Go SDK. Invokers are memoized and when possible code
>> is generated to avoid reflection.
>> >> >> >
>> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com>
>> wrote:
>> >> >> >>
>> >> >> >> Noting for the benefit of the thread archive in case someone
>> goes digging and wonders if this affects other SDKs: the Java SDK memoizes
>> DoFnSignatures and generated DoFnInvoker classes.
>> >> >> >>
>> >> >> >> Kenn
>> >> >> >>
>> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com>
>> wrote:
>> >> >> >>>
>> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>> >> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
>> >> >> >>> 2. The PR somehow changed how coders are selected, and now
>> we're using less efficient ones.
>> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>> >> >> >>>
>> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and
>> 3.7 and got these cool graphs (attached).
>> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time
>> (IIUC), gets called ~230k times
>> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k
>> times
>> >> >> >>>
>> >> >> >>> After memoization of get_function_args_defaults:
>> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
>> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
>> >> >> >>>
>> >> >> >>>
>> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >> >>>>
>> >> >> >>>> *not deciles, but 9-percentiles : )
>> >> >> >>>>
>> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >> >>>>>
>> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a
>> utility to track runtime for each test method. I found some of the
>> following things:
>> >> >> >>>>> - Total test methods run: 2665
>> >> >> >>>>> - Total test runtime: 990 seconds
>> >> >> >>>>> - Deciles of time spent:
>> >> >> >>>>>   - 1949 tests run in the first 9% of time
>> >> >> >>>>>   - 173 in the 9-18% rang3e
>> >> >> >>>>>   - 130 in the 18-27% range
>> >> >> >>>>>   - 95 in the 27-36% range
>> >> >> >>>>>   - 77
>> >> >> >>>>>   - 66
>> >> >> >>>>>   - 55
>> >> >> >>>>>   - 46
>> >> >> >>>>>   - 37
>> >> >> >>>>>   - 24
>> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents
>> about 1 minute and a half.
>> >> >> >>>>>
>> >> >> >>>>> We may be able to look at the slowest X tests, and get
>> gradual improvements from there. Although it seems .. not dramatic ones : )
>> >> >> >>>>>
>> >> >> >>>>> FWIW I uploaded the results here:
>> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>> >> >> >>>>>
>> >> >> >>>>> The slowest 13 tests were:
>> >> >> >>>>>
>> >> >> >>>>>
>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>> >> >> >>>>>   5.253582000732422),
>> >> >> >>>>>
>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>> >> >> >>>>>   7.907713890075684),
>> >> >> >>>>>
>> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>> >> >> >>>>>   5.237942934036255),
>> >> >> >>>>>
>> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>> >> >> >>>>>   5.563946008682251),
>> >> >> >>>>>
>> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>> >> >> >>>>>   5.680700063705444),
>> >> >> >>>>>
>> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>> >> >> >>>>>   6.111238956451416),
>> >> >> >>>>>
>> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>> >> >> >>>>>   6.007534980773926),
>> >> >> >>>>>
>> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>> >> >> >>>>>   13.993916988372803),
>> >> >> >>>>>
>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>> >> >> >>>>>   6.3383049964904785),
>> >> >> >>>>>
>> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>> >> >> >>>>>   9.157485008239746),
>> >> >> >>>>>
>> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>> >> >> >>>>>   5.191173076629639),
>> >> >> >>>>>
>> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>> >> >> >>>>>   6.2221620082855225),
>> >> >> >>>>>
>> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>> >> >> >>>>>   7.7187910079956055)]
>> >> >> >>>>>
>> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >> >>>>>>
>> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to
>> reduce FnApiRunnerTest variations.
>> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a
>> discussion.
>> >> >> >>>>>> I'll also try to figure out if there are other tests slowing
>> down the suite significantly.
>> >> >> >>>>>> Best
>> >> >> >>>>>> -P.
>> >> >> >>>>>>
>> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
>> valentyn@google.com> wrote:
>> >> >> >>>>>>>
>> >> >> >>>>>>> Thanks, Brian.
>> >> >> >>>>>>> +Udi Meiri
>> >> >> >>>>>>> As next step, it would be good to know whether slowdown is
>> caused by tests in this PR, or its effect on other tests, and to confirm
>> that only Python 2 codepaths were affected.
>> >> >> >>>>>>>
>> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
>> bhulette@google.com> wrote:
>> >> >> >>>>>>>>
>> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
>> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
>> and 9/15 to see if I could find the source of the spike that happened
>> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
>> search would reveal some mis-guided configuration change, but as far as I
>> can tell 9283 just added a well-tested feature. I don't think there's
>> anything to learn from that... I just wanted to circle back about it in
>> case others are curious about that spike.
>> >> >> >>>>>>>>
>> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>> >> >> >>>>>>>>
>> >> >> >>>>>>>> Brian
>> >> >> >>>>>>>>
>> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>> >> >> >>>>>>>>
>> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
>> pabloem@google.com> wrote:
>> >> >> >>>>>>>>>
>> >> >> >>>>>>>>> I think it makes sense to remove some of the extra
>> FnApiRunner configurations. Perhaps some of the multiworkers and some of
>> the grpc versions?
>> >> >> >>>>>>>>> Best
>> >> >> >>>>>>>>> -P.
>> >> >> >>>>>>>>>
>> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
>> robertwb@google.com> wrote:
>> >> >> >>>>>>>>>>
>> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive,
>> taking 10-15+
>> >> >> >>>>>>>>>> minutes on each version of Python. This test consists of
>> a base class
>> >> >> >>>>>>>>>> that is basically a validates runner suite, and is then
>> run in several
>> >> >> >>>>>>>>>> configurations, many more of which (including some
>> expensive ones)
>> >> >> >>>>>>>>>> have been added lately.
>> >> >> >>>>>>>>>>
>> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>> >> >> >>>>>>>>>> class
>> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>> >> >> >>>>>>>>>>
>> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
>> permutations, or at
>> >> >> >>>>>>>>>> least not all tests in all permutations.
>> >> >> >>>>>>>>>>
>> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>> >> >> >>>>>>>>>> <va...@google.com> wrote:
>> >> >> >>>>>>>>>> >
>> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are
>> already running in parallel, albeit in the same suite. However it appears
>> Python precommits became slower, especially Python 2 precommits [35 min per
>> suite x 3 suites], see [1]. Not sure yet what caused the increase, but
>> precommits used to be faster. Perhaps we have added a slow test or a lot of
>> new tests.
>> >> >> >>>>>>>>>> >
>> >> >> >>>>>>>>>> > [1]
>> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>> >> >> >>>>>>>>>> >
>> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
>> altay@google.com> wrote:
>> >> >> >>>>>>>>>> >>
>> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite
>> sounds good. Anyone is interested in doing that?
>> >> >> >>>>>>>>>> >>
>> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <
>> valentyn@google.com> wrote:
>> >> >> >>>>>>>>>> >>>
>> >> >> >>>>>>>>>> >>> This should not increase the queue time
>> substantially, since precommit ITs are running sequentially with precommit
>> tests, unlike multiple precommit tests which run in parallel to each other.
>> >> >> >>>>>>>>>> >>>
>> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming
>> wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>> >> >> >>>>>>>>>> >>>
>> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
>> altay@google.com> wrote:
>> >> >> >>>>>>>>>> >>>>
>> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would
>> be, when Chad tried to do something similar [1] it was noted that the total
>> time to run all precommit tests would increase and also potentially
>> increase the queue time.
>> >> >> >>>>>>>>>> >>>>
>> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of
>> IT tests in precommits and run the whole suite as part of post commit tests.
>> >> >> >>>>>>>>>> >>>>
>> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>> >> >> >>>>>>>>>> >>>>
>> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn
>> Tymofieiev <va...@google.com> wrote:
>> >> >> >>>>>>>>>> >>>>>
>> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT
>> tests into a separate suite from precommit tests, and run it in parallel.
>> >> >> >>>>>>>>>> >>>>>
>> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <
>> bhulette@google.com> wrote:
>> >> >> >>>>>>>>>> >>>>>>
>> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now
>> [1]. Just visually it looks like the average length is 1.5h or so, but it
>> spikes up to 2h. I've had several precommit runs get aborted due to the 2
>> hour limit.
>> >> >> >>>>>>>>>> >>>>>>
>> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back
>> on 9/6 and the duration has been steadily rising since then. Is there
>> anything we can do about this?
>> >> >> >>>>>>>>>> >>>>>>
>> >> >> >>>>>>>>>> >>>>>> Brian
>> >> >> >>>>>>>>>> >>>>>>
>> >> >> >>>>>>>>>> >>>>>> [1]
>> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>>
>

Re: Python Precommit duration pushing 2 hours

Posted by Udi Meiri <eh...@google.com>.
+1

On Mon, Nov 4, 2019 at 12:09 PM Robert Bradshaw <ro...@google.com> wrote:

> +1, this seems like a good step with a clear win.
>
> On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
> >
> > Python precommits are still timing out on #9925. I am guessing that
> means this change would not be enough.
> >
> > I am proposing cutting down the number of test variants we run in
> precommits. Currently for each version we ran the following variants
> serially:
> > - base: Runs all unit tests with tox
> > - Cython: Installs cython and runs all unit tests as base version. The
> original purpose was to ensure that tests pass with or without cython.
> There is probably a huge overlap with base. (IIRC only a few coders have
> different slow vs fast tests.)
> > - GCP: Installs GCP dependencies and tests all base + additional gcp
> specific tests. The original purpose was to ensure that GCP is an optional
> component and all non-GCP tests still works without GCP components.
> >
> > We can reduce the list to cython + GCP tests only. This will cover the
> same group of tests and will check that tests pass with or without cython
> or GCP dependencies. This could reduce the precommit time by ~30 minutes.
> >
> > What do you think?
> >
> > Ahmet
> >
> >
> > On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >>
> >> https://github.com/apache/beam/pull/9925
> >>
> >> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
> >> >
> >> > I don't have the bandwidth right now to tackle this. Feel free to
> take it.
> >> >
> >> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com>
> wrote:
> >> >>
> >> >> The Python SDK does as well. These calls are coming from
> >> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which are
> >> >> invoked once per pipene or bundle. They are, however, surprisingly
> >> >> expensive. Even memoizing across those three calls should save a
> >> >> significant amount of time. Udi, did you want to tackle this?
> >> >>
> >> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
> >> >> times in this test, and [Applied]PTransform.to_fn_api being called
> >> >> 3111 times, so that in itself might be interesting to investigate.
> >> >>
> >> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com>
> wrote:
> >> >> >
> >> >> > As does the Go SDK. Invokers are memoized and when possible code
> is generated to avoid reflection.
> >> >> >
> >> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com>
> wrote:
> >> >> >>
> >> >> >> Noting for the benefit of the thread archive in case someone goes
> digging and wonders if this affects other SDKs: the Java SDK memoizes
> DoFnSignatures and generated DoFnInvoker classes.
> >> >> >>
> >> >> >> Kenn
> >> >> >>
> >> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com>
> wrote:
> >> >> >>>
> >> >> >>> Re: #9283 slowing down tests, ideas for slowness:
> >> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
> >> >> >>> 2. The PR somehow changed how coders are selected, and now we're
> using less efficient ones.
> >> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
> >> >> >>>
> >> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and
> 3.7 and got these cool graphs (attached).
> >> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time
> (IIUC), gets called ~230k times
> >> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k
> times
> >> >> >>>
> >> >> >>> After memoization of get_function_args_defaults:
> >> >> >>> 2.7: core:294:get_function_arguments 20.02%
> >> >> >>> 3.7: core:294:get_function_arguments 8.11%
> >> >> >>>
> >> >> >>>
> >> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >> >>>>
> >> >> >>>> *not deciles, but 9-percentiles : )
> >> >> >>>>
> >> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >> >>>>>
> >> >> >>>>> I've ran the tests in Python 2 (without cython), and used a
> utility to track runtime for each test method. I found some of the
> following things:
> >> >> >>>>> - Total test methods run: 2665
> >> >> >>>>> - Total test runtime: 990 seconds
> >> >> >>>>> - Deciles of time spent:
> >> >> >>>>>   - 1949 tests run in the first 9% of time
> >> >> >>>>>   - 173 in the 9-18% rang3e
> >> >> >>>>>   - 130 in the 18-27% range
> >> >> >>>>>   - 95 in the 27-36% range
> >> >> >>>>>   - 77
> >> >> >>>>>   - 66
> >> >> >>>>>   - 55
> >> >> >>>>>   - 46
> >> >> >>>>>   - 37
> >> >> >>>>>   - 24
> >> >> >>>>>   - 13 tests run in the last 9% of time. This represents about
> 1 minute and a half.
> >> >> >>>>>
> >> >> >>>>> We may be able to look at the slowest X tests, and get gradual
> improvements from there. Although it seems .. not dramatic ones : )
> >> >> >>>>>
> >> >> >>>>> FWIW I uploaded the results here:
> https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
> >> >> >>>>>
> >> >> >>>>> The slowest 13 tests were:
> >> >> >>>>>
> >> >> >>>>>
> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
> >> >> >>>>>   5.253582000732422),
> >> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
> >> >> >>>>>   7.907713890075684),
> >> >> >>>>>
> ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
> >> >> >>>>>   5.237942934036255),
> >> >> >>>>>
> ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
> >> >> >>>>>   5.563946008682251),
> >> >> >>>>>
> ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
> >> >> >>>>>   5.680700063705444),
> >> >> >>>>>
> ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
> >> >> >>>>>   6.111238956451416),
> >> >> >>>>>
> ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
> >> >> >>>>>   6.007534980773926),
> >> >> >>>>>
> ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
> >> >> >>>>>   13.993916988372803),
> >> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
> >> >> >>>>>   6.3383049964904785),
> >> >> >>>>>
> ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
> >> >> >>>>>   9.157485008239746),
> >> >> >>>>>
> ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
> >> >> >>>>>   5.191173076629639),
> >> >> >>>>>
> ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
> >> >> >>>>>   6.2221620082855225),
> >> >> >>>>>
> ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
> >> >> >>>>>   7.7187910079956055)]
> >> >> >>>>>
> >> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >> >>>>>>
> >> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to
> reduce FnApiRunnerTest variations.
> >> >> >>>>>> I'm not in a rush to merge, but rather happy to start a
> discussion.
> >> >> >>>>>> I'll also try to figure out if there are other tests slowing
> down the suite significantly.
> >> >> >>>>>> Best
> >> >> >>>>>> -P.
> >> >> >>>>>>
> >> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <
> valentyn@google.com> wrote:
> >> >> >>>>>>>
> >> >> >>>>>>> Thanks, Brian.
> >> >> >>>>>>> +Udi Meiri
> >> >> >>>>>>> As next step, it would be good to know whether slowdown is
> caused by tests in this PR, or its effect on other tests, and to confirm
> that only Python 2 codepaths were affected.
> >> >> >>>>>>>
> >> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <
> bhulette@google.com> wrote:
> >> >> >>>>>>>>
> >> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew
> :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1
> and 9/15 to see if I could find the source of the spike that happened
> around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this
> search would reveal some mis-guided configuration change, but as far as I
> can tell 9283 just added a well-tested feature. I don't think there's
> anything to learn from that... I just wanted to circle back about it in
> case others are curious about that spike.
> >> >> >>>>>>>>
> >> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
> >> >> >>>>>>>>
> >> >> >>>>>>>> Brian
> >> >> >>>>>>>>
> >> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
> >> >> >>>>>>>>
> >> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <
> pabloem@google.com> wrote:
> >> >> >>>>>>>>>
> >> >> >>>>>>>>> I think it makes sense to remove some of the extra
> FnApiRunner configurations. Perhaps some of the multiworkers and some of
> the grpc versions?
> >> >> >>>>>>>>> Best
> >> >> >>>>>>>>> -P.
> >> >> >>>>>>>>>
> >> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <
> robertwb@google.com> wrote:
> >> >> >>>>>>>>>>
> >> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive,
> taking 10-15+
> >> >> >>>>>>>>>> minutes on each version of Python. This test consists of
> a base class
> >> >> >>>>>>>>>> that is basically a validates runner suite, and is then
> run in several
> >> >> >>>>>>>>>> configurations, many more of which (including some
> expensive ones)
> >> >> >>>>>>>>>> have been added lately.
> >> >> >>>>>>>>>>
> >> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
> >> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
> >> >> >>>>>>>>>> class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
> >> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
> >> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
> >> >> >>>>>>>>>> class
> FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
> >> >> >>>>>>>>>>
> >> >> >>>>>>>>>> I'm not convinced we need to run all of these
> permutations, or at
> >> >> >>>>>>>>>> least not all tests in all permutations.
> >> >> >>>>>>>>>>
> >> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
> >> >> >>>>>>>>>> <va...@google.com> wrote:
> >> >> >>>>>>>>>> >
> >> >> >>>>>>>>>> > I took another look at this and precommit ITs are
> already running in parallel, albeit in the same suite. However it appears
> Python precommits became slower, especially Python 2 precommits [35 min per
> suite x 3 suites], see [1]. Not sure yet what caused the increase, but
> precommits used to be faster. Perhaps we have added a slow test or a lot of
> new tests.
> >> >> >>>>>>>>>> >
> >> >> >>>>>>>>>> > [1]
> https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
> >> >> >>>>>>>>>> >
> >> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <
> altay@google.com> wrote:
> >> >> >>>>>>>>>> >>
> >> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite
> sounds good. Anyone is interested in doing that?
> >> >> >>>>>>>>>> >>
> >> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <
> valentyn@google.com> wrote:
> >> >> >>>>>>>>>> >>>
> >> >> >>>>>>>>>> >>> This should not increase the queue time
> substantially, since precommit ITs are running sequentially with precommit
> tests, unlike multiple precommit tests which run in parallel to each other.
> >> >> >>>>>>>>>> >>>
> >> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming
> wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
> >> >> >>>>>>>>>> >>>
> >> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <
> altay@google.com> wrote:
> >> >> >>>>>>>>>> >>>>
> >> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would
> be, when Chad tried to do something similar [1] it was noted that the total
> time to run all precommit tests would increase and also potentially
> increase the queue time.
> >> >> >>>>>>>>>> >>>>
> >> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of
> IT tests in precommits and run the whole suite as part of post commit tests.
> >> >> >>>>>>>>>> >>>>
> >> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
> >> >> >>>>>>>>>> >>>>
> >> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn Tymofieiev
> <va...@google.com> wrote:
> >> >> >>>>>>>>>> >>>>>
> >> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT tests
> into a separate suite from precommit tests, and run it in parallel.
> >> >> >>>>>>>>>> >>>>>
> >> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <
> bhulette@google.com> wrote:
> >> >> >>>>>>>>>> >>>>>>
> >> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now
> [1]. Just visually it looks like the average length is 1.5h or so, but it
> spikes up to 2h. I've had several precommit runs get aborted due to the 2
> hour limit.
> >> >> >>>>>>>>>> >>>>>>
> >> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back
> on 9/6 and the duration has been steadily rising since then. Is there
> anything we can do about this?
> >> >> >>>>>>>>>> >>>>>>
> >> >> >>>>>>>>>> >>>>>> Brian
> >> >> >>>>>>>>>> >>>>>>
> >> >> >>>>>>>>>> >>>>>> [1]
> http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4
>

Re: Python Precommit duration pushing 2 hours

Posted by Robert Bradshaw <ro...@google.com>.
+1, this seems like a good step with a clear win.

On Mon, Nov 4, 2019 at 12:06 PM Ahmet Altay <al...@google.com> wrote:
>
> Python precommits are still timing out on #9925. I am guessing that means this change would not be enough.
>
> I am proposing cutting down the number of test variants we run in precommits. Currently for each version we ran the following variants serially:
> - base: Runs all unit tests with tox
> - Cython: Installs cython and runs all unit tests as base version. The original purpose was to ensure that tests pass with or without cython. There is probably a huge overlap with base. (IIRC only a few coders have different slow vs fast tests.)
> - GCP: Installs GCP dependencies and tests all base + additional gcp specific tests. The original purpose was to ensure that GCP is an optional component and all non-GCP tests still works without GCP components.
>
> We can reduce the list to cython + GCP tests only. This will cover the same group of tests and will check that tests pass with or without cython or GCP dependencies. This could reduce the precommit time by ~30 minutes.
>
> What do you think?
>
> Ahmet
>
>
> On Tue, Oct 29, 2019 at 11:15 AM Robert Bradshaw <ro...@google.com> wrote:
>>
>> https://github.com/apache/beam/pull/9925
>>
>> On Tue, Oct 29, 2019 at 10:24 AM Udi Meiri <eh...@google.com> wrote:
>> >
>> > I don't have the bandwidth right now to tackle this. Feel free to take it.
>> >
>> > On Tue, Oct 29, 2019 at 10:16 AM Robert Bradshaw <ro...@google.com> wrote:
>> >>
>> >> The Python SDK does as well. These calls are coming from
>> >> to_runner_api, is_stateful_dofn, and validate_stateful_dofn which are
>> >> invoked once per pipene or bundle. They are, however, surprisingly
>> >> expensive. Even memoizing across those three calls should save a
>> >> significant amount of time. Udi, did you want to tackle this?
>> >>
>> >> Looking at the profile, Pipeline.to_runner_api() is being called 30
>> >> times in this test, and [Applied]PTransform.to_fn_api being called
>> >> 3111 times, so that in itself might be interesting to investigate.
>> >>
>> >> On Tue, Oct 29, 2019 at 8:26 AM Robert Burke <ro...@frantil.com> wrote:
>> >> >
>> >> > As does the Go SDK. Invokers are memoized and when possible code is generated to avoid reflection.
>> >> >
>> >> > On Tue, Oct 29, 2019, 6:46 AM Kenneth Knowles <kl...@google.com> wrote:
>> >> >>
>> >> >> Noting for the benefit of the thread archive in case someone goes digging and wonders if this affects other SDKs: the Java SDK memoizes DoFnSignatures and generated DoFnInvoker classes.
>> >> >>
>> >> >> Kenn
>> >> >>
>> >> >> On Mon, Oct 28, 2019 at 6:59 PM Udi Meiri <eh...@google.com> wrote:
>> >> >>>
>> >> >>> Re: #9283 slowing down tests, ideas for slowness:
>> >> >>> 1. I added a lot of test cases, some with locally run pipelines.
>> >> >>> 2. The PR somehow changed how coders are selected, and now we're using less efficient ones.
>> >> >>> 3. New dependency funcsigs is slowing things down? (py2 only)
>> >> >>>
>> >> >>> I ran "pytest -k PipelineAnalyzerTest --profile-svg" on 2.7 and 3.7 and got these cool graphs (attached).
>> >> >>> 2.7: core:294:get_function_arguments takes 56.66% of CPU time (IIUC), gets called ~230k times
>> >> >>> 3.7: core:294:get_function_arguments 30.88%, gets called ~200k times
>> >> >>>
>> >> >>> After memoization of get_function_args_defaults:
>> >> >>> 2.7: core:294:get_function_arguments 20.02%
>> >> >>> 3.7: core:294:get_function_arguments 8.11%
>> >> >>>
>> >> >>>
>> >> >>> On Mon, Oct 28, 2019 at 5:38 PM Pablo Estrada <pa...@google.com> wrote:
>> >> >>>>
>> >> >>>> *not deciles, but 9-percentiles : )
>> >> >>>>
>> >> >>>> On Mon, Oct 28, 2019 at 5:31 PM Pablo Estrada <pa...@google.com> wrote:
>> >> >>>>>
>> >> >>>>> I've ran the tests in Python 2 (without cython), and used a utility to track runtime for each test method. I found some of the following things:
>> >> >>>>> - Total test methods run: 2665
>> >> >>>>> - Total test runtime: 990 seconds
>> >> >>>>> - Deciles of time spent:
>> >> >>>>>   - 1949 tests run in the first 9% of time
>> >> >>>>>   - 173 in the 9-18% rang3e
>> >> >>>>>   - 130 in the 18-27% range
>> >> >>>>>   - 95 in the 27-36% range
>> >> >>>>>   - 77
>> >> >>>>>   - 66
>> >> >>>>>   - 55
>> >> >>>>>   - 46
>> >> >>>>>   - 37
>> >> >>>>>   - 24
>> >> >>>>>   - 13 tests run in the last 9% of time. This represents about 1 minute and a half.
>> >> >>>>>
>> >> >>>>> We may be able to look at the slowest X tests, and get gradual improvements from there. Although it seems .. not dramatic ones : )
>> >> >>>>>
>> >> >>>>> FWIW I uploaded the results here: https://storage.googleapis.com/apache-beam-website-pull-requests/python-tests/nosetimes.json
>> >> >>>>>
>> >> >>>>> The slowest 13 tests were:
>> >> >>>>>
>> >> >>>>> [('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_basic',
>> >> >>>>>   5.253582000732422),
>> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_wordcount',
>> >> >>>>>   7.907713890075684),
>> >> >>>>>  ('apache_beam.io.gcp.bigquery_test.PipelineBasedStreamingInsertTest.test_failure_has_same_insert_ids',
>> >> >>>>>   5.237942934036255),
>> >> >>>>>  ('apache_beam.transforms.combiners_test.CombineTest.test_global_sample',
>> >> >>>>>   5.563946008682251),
>> >> >>>>>  ('apache_beam.runners.worker.sideinputs_test.EmulatedCollectionsTest.test_large_iterable_values',
>> >> >>>>>   5.680700063705444),
>> >> >>>>>  ('apache_beam.io.parquetio_test.TestParquet.test_sink_transform_multiple_row_group',
>> >> >>>>>   6.111238956451416),
>> >> >>>>>  ('apache_beam.runners.worker.statesampler_test.StateSamplerTest.test_basic_sampler',
>> >> >>>>>   6.007534980773926),
>> >> >>>>>  ('apache_beam.runners.interactive.interactive_runner_test.InteractiveRunnerTest.test_basic',
>> >> >>>>>   13.993916988372803),
>> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_read_cache_expansion',
>> >> >>>>>   6.3383049964904785),
>> >> >>>>>  ('apache_beam.runners.interactive.pipeline_analyzer_test.PipelineAnalyzerTest.test_word_count',
>> >> >>>>>   9.157485008239746),
>> >> >>>>>  ('apache_beam.runners.portability.portable_runner_test.PortableRunnerTestWithSubprocesses.test_pardo_side_and_main_outputs',
>> >> >>>>>   5.191173076629639),
>> >> >>>>>  ('apache_beam.io.vcfio_test.VcfSourceTest.test_pipeline_read_file_pattern_large',
>> >> >>>>>   6.2221620082855225),
>> >> >>>>>  ('apache_beam.io.fileio_test.WriteFilesTest.test_streaming_complex_timing',
>> >> >>>>>   7.7187910079956055)]
>> >> >>>>>
>> >> >>>>> On Mon, Oct 28, 2019 at 3:10 PM Pablo Estrada <pa...@google.com> wrote:
>> >> >>>>>>
>> >> >>>>>> I have written https://github.com/apache/beam/pull/9910 to reduce FnApiRunnerTest variations.
>> >> >>>>>> I'm not in a rush to merge, but rather happy to start a discussion.
>> >> >>>>>> I'll also try to figure out if there are other tests slowing down the suite significantly.
>> >> >>>>>> Best
>> >> >>>>>> -P.
>> >> >>>>>>
>> >> >>>>>> On Fri, Oct 25, 2019 at 7:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>> >> >>>>>>>
>> >> >>>>>>> Thanks, Brian.
>> >> >>>>>>> +Udi Meiri
>> >> >>>>>>> As next step, it would be good to know whether slowdown is caused by tests in this PR, or its effect on other tests, and to confirm that only Python 2 codepaths were affected.
>> >> >>>>>>>
>> >> >>>>>>> On Fri, Oct 25, 2019 at 6:35 PM Brian Hulette <bh...@google.com> wrote:
>> >> >>>>>>>>
>> >> >>>>>>>> I did a bisect based on the runtime of `./gradlew :sdks:python:test-suites:tox:py2:testPy2Gcp` around the commits between 9/1 and 9/15 to see if I could find the source of the spike that happened around 9/6. It looks like it was due to PR#9283 [1]. I thought maybe this search would reveal some mis-guided configuration change, but as far as I can tell 9283 just added a well-tested feature. I don't think there's anything to learn from that... I just wanted to circle back about it in case others are curious about that spike.
>> >> >>>>>>>>
>> >> >>>>>>>> I'm +1 on bumping some FnApiRunner configurations.
>> >> >>>>>>>>
>> >> >>>>>>>> Brian
>> >> >>>>>>>>
>> >> >>>>>>>> [1] https://github.com/apache/beam/pull/9283
>> >> >>>>>>>>
>> >> >>>>>>>> On Fri, Oct 25, 2019 at 4:49 PM Pablo Estrada <pa...@google.com> wrote:
>> >> >>>>>>>>>
>> >> >>>>>>>>> I think it makes sense to remove some of the extra FnApiRunner configurations. Perhaps some of the multiworkers and some of the grpc versions?
>> >> >>>>>>>>> Best
>> >> >>>>>>>>> -P.
>> >> >>>>>>>>>
>> >> >>>>>>>>> On Fri, Oct 25, 2019 at 12:27 PM Robert Bradshaw <ro...@google.com> wrote:
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> It looks like fn_api_runner_test.py is quite expensive, taking 10-15+
>> >> >>>>>>>>>> minutes on each version of Python. This test consists of a base class
>> >> >>>>>>>>>> that is basically a validates runner suite, and is then run in several
>> >> >>>>>>>>>> configurations, many more of which (including some expensive ones)
>> >> >>>>>>>>>> have been added lately.
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> class FnApiRunnerTest(unittest.TestCase):
>> >> >>>>>>>>>> class FnApiRunnerTestWithGrpc(FnApiRunnerTest):
>> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcMultiThreaded(FnApiRunnerTest):
>> >> >>>>>>>>>> class FnApiRunnerTestWithDisabledCaching(FnApiRunnerTest):
>> >> >>>>>>>>>> class FnApiRunnerTestWithMultiWorkers(FnApiRunnerTest):
>> >> >>>>>>>>>> class FnApiRunnerTestWithGrpcAndMultiWorkers(FnApiRunnerTest):
>> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeat(FnApiRunnerTest):
>> >> >>>>>>>>>> class FnApiRunnerTestWithBundleRepeatAndMultiWorkers(FnApiRunnerTest):
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> I'm not convinced we need to run all of these permutations, or at
>> >> >>>>>>>>>> least not all tests in all permutations.
>> >> >>>>>>>>>>
>> >> >>>>>>>>>> On Fri, Oct 25, 2019 at 10:57 AM Valentyn Tymofieiev
>> >> >>>>>>>>>> <va...@google.com> wrote:
>> >> >>>>>>>>>> >
>> >> >>>>>>>>>> > I took another look at this and precommit ITs are already running in parallel, albeit in the same suite. However it appears Python precommits became slower, especially Python 2 precommits [35 min per suite x 3 suites], see [1]. Not sure yet what caused the increase, but precommits used to be faster. Perhaps we have added a slow test or a lot of new tests.
>> >> >>>>>>>>>> >
>> >> >>>>>>>>>> > [1] https://scans.gradle.com/s/jvcw5fpqfc64k/timeline?task=ancsbov425524
>> >> >>>>>>>>>> >
>> >> >>>>>>>>>> > On Thu, Oct 24, 2019 at 4:53 PM Ahmet Altay <al...@google.com> wrote:
>> >> >>>>>>>>>> >>
>> >> >>>>>>>>>> >> Ack. Separating precommit ITs to a different suite sounds good. Anyone is interested in doing that?
>> >> >>>>>>>>>> >>
>> >> >>>>>>>>>> >> On Thu, Oct 24, 2019 at 2:41 PM Valentyn Tymofieiev <va...@google.com> wrote:
>> >> >>>>>>>>>> >>>
>> >> >>>>>>>>>> >>> This should not increase the queue time substantially, since precommit ITs are running sequentially with precommit tests, unlike multiple precommit tests which run in parallel to each other.
>> >> >>>>>>>>>> >>>
>> >> >>>>>>>>>> >>> The precommit ITs we run are batch and streaming wordcount tests on Py2 and one Py3 version, so it's not a lot of tests.
>> >> >>>>>>>>>> >>>
>> >> >>>>>>>>>> >>> On Thu, Oct 24, 2019 at 1:07 PM Ahmet Altay <al...@google.com> wrote:
>> >> >>>>>>>>>> >>>>
>> >> >>>>>>>>>> >>>> +1 to separating ITs from precommit. Downside would be, when Chad tried to do something similar [1] it was noted that the total time to run all precommit tests would increase and also potentially increase the queue time.
>> >> >>>>>>>>>> >>>>
>> >> >>>>>>>>>> >>>> Another alternative, we could run a smaller set of IT tests in precommits and run the whole suite as part of post commit tests.
>> >> >>>>>>>>>> >>>>
>> >> >>>>>>>>>> >>>> [1] https://github.com/apache/beam/pull/9642
>> >> >>>>>>>>>> >>>>
>> >> >>>>>>>>>> >>>> On Thu, Oct 24, 2019 at 12:15 PM Valentyn Tymofieiev <va...@google.com> wrote:
>> >> >>>>>>>>>> >>>>>
>> >> >>>>>>>>>> >>>>> One improvement could be move to Precommit IT tests into a separate suite from precommit tests, and run it in parallel.
>> >> >>>>>>>>>> >>>>>
>> >> >>>>>>>>>> >>>>> On Thu, Oct 24, 2019 at 11:41 AM Brian Hulette <bh...@google.com> wrote:
>> >> >>>>>>>>>> >>>>>>
>> >> >>>>>>>>>> >>>>>> Python Precommits are taking quite a while now [1]. Just visually it looks like the average length is 1.5h or so, but it spikes up to 2h. I've had several precommit runs get aborted due to the 2 hour limit.
>> >> >>>>>>>>>> >>>>>>
>> >> >>>>>>>>>> >>>>>> It looks like there was a spike up above 1h back on 9/6 and the duration has been steadily rising since then. Is there anything we can do about this?
>> >> >>>>>>>>>> >>>>>>
>> >> >>>>>>>>>> >>>>>> Brian
>> >> >>>>>>>>>> >>>>>>
>> >> >>>>>>>>>> >>>>>> [1] http://104.154.241.245/d/_TNndF2iz/pre-commit-test-latency?orgId=1&from=now-90d&to=now&fullscreen&panelId=4