You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@beam.apache.org by GitBox <gi...@apache.org> on 2022/07/14 21:24:39 UTC

[GitHub] [beam] Abacn opened a new issue, #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Abacn opened a new issue, #22283:
URL: https://github.com/apache/beam/issues/22283

   ### What happened?
   
   Was investigating #22115 and find this interesting observation
   
   pytest results with time consuming info:
   
   ```
   ----------------------------------
   Original:
   =========================================================================================== slowest 100 test durations ============================================================================================
   10.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pack_combiners
   9.82s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pack_combiners
   5.41s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_side_and_main_outputs
   5.20s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_outputs
   5.20s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_to_element_pardo
   5.17s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_unfusable_side_inputs_with_separation
   5.16s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_callbacks_with_exception
   5.16s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo_overlapping_windows
   5.16s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_draining_sdf_with_sdf_initiated_checkpointing
   5.15s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo_fusion_break
   5.14s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_flatten_same_pcollections
   5.14s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint_sdf
   5.13s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo
   5.13s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_assert_that
   5.13s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint
   5.13s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo_window_param
   5.12s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_element_to_batch_pardo
   5.12s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo_override_type_inference
   5.11s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_override_type_inference
   5.11s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo_dofn_params
   5.09s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_create
   5.09s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_assert_that
   5.09s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_crazy_draining_sdf
   5.09s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_crazy_sdf
   5.04s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_custom_merging_window
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_timers_clear
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_inputs
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_sdf_with_sdf_initiated_checkpointing
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf_default_truncate_when_bounded
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_read
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_half_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_flatten
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_half_draining_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_error_traceback_includes_user_code
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_nosplit_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_sdf_with_truncate
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_state_timers_non_standard_coder
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_state_only
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_sdf_with_watermark_tracking
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_gbk_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_side_input_dependencies
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_unfusable_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pack_combiners
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_combine_per_key
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf_with_dofn_as_restriction_provider
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_sdf_with_sdf_initiated_checkpointing
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_state_timers
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf_with_check_done_failed
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_create_value_provider_pipeline_option
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_fusion_break
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_error_traceback_includes_user_code
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_window_param
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_dofn_params
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_gbk_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_batch_pardo
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_batch_pardo_overlapping_windows
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_metrics
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_timers_clear
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_sdf_with_watermark_tracking
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_sdf_with_truncate
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_windowed_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_and_main_outputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_flatten
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_side_outputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf_with_truncate
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_combine_per_key
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_callbacks_with_exception
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_no_subtransform_composite
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_state_only
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_group_by_key
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_flattened_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_batch_pardo_dofn_params
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_windowed_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_error_message_includes_stage
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_windowed_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_combine_per_key
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_state_timers_non_standard_coder
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_multimap_side_input_type_coercion
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_state_timers
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_and_main_outputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_timers
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_dynamic_timer
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_flattened_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_overlapping_windows
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_sdf_default_truncate_when_unbounded
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_flatten_same_pcollections
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_state_timers
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_sdf_synthetic_source
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_flattened_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_metrics
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_batch_pardo_window_param
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_input_dependencies
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_multimap_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_state_only
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_error_message_includes_stage
   ============================================================================== 407 passed, 54 skipped, 24 warnings in 88.40 seconds ===============================================================================
   ```
   
   There is a default 5 seconds to close the rpc server in https://github.com/apache/beam/blob/c14a4cf877908852c139f67c0c28218c1339269c/sdks/python/apache_beam/runners/portability/fn_api_runner/worker_handlers.py#L453 
   Change this to None (or mock this constant in tests), tests still pass and running time decrease dramatically:
   ```
   ============================================================================================ slowest 20 test durations ============================================================================================
   5.25s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint_sdf
   5.15s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_half_draining_sdf
   5.13s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint
   5.12s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_crazy_sdf
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_nosplit_sdf
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_crazy_draining_sdf
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_half_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_half
   1.48s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pack_combiners
   1.44s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_and_main_outputs
   1.38s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint_draining_sdf
   1.19s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_register_finalizations
   1.18s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_register_finalizations
   1.17s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTest::test_register_finalizations
   1.15s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithBundleRepeatAndMultiWorkers::test_pardo_side_and_main_outputs
   1.14s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithBundleRepeatAndMultiWorkers::test_pack_combiners
   1.06s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithMultiWorkers::test_pardo_side_and_main_outputs
   0.93s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithMultiWorkers::test_pack_combiners
   0.82s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_side_and_main_outputs
   0.74s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pack_combiners
   ============================================================================== 407 passed, 54 skipped, 24 warnings in 22.88 seconds ===============================================================================
   ```
   
   Suprisingly, if changing `_DEFAULT_SHUTDOWN_TIMEOUT_SECS` to a large number (100), the running time is IDENTICAL with setting it as 5 seconds---hundreds of tests costs exactly 5 seconds, likely there is a hard-coded waiting time of 5 seconds that prevented RpcServer from closing.
   
   ### Issue Priority
   
   Priority: 2
   
   ### Issue Component
   
   Component: testing


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205694644

   Changing the priority. This is blocking the release according to the release email. 
   
   Could we rollback to a previous grpc version to unblock the release? 


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] pabloem commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1211575508

   I may not be able to look at this for now : /


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] tvalentyn commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
tvalentyn commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205338266

   Or is the issue that the test is waiting to shutdown the servers after completing execution of the scenario?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1213648838

   I assume this is no longer blocking the release, given that we have an RC ?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kennknowles commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
kennknowles commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1246038490

   If not blocking the release and not considered a performance regression, lowering to P2.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1191837062

   Found this issue is introduced in grpcio-1.46.0. Nailed this down to changes added to grpc master between March 21 to March 22, most likely this commit: https://github.com/grpc/grpc/pull/29050
   
   Also found the server that does not shutdown before grace time is mostly state_server; and control_server also prevents shutdown in some tests:
   
   no graceful shutsown for control_server alone:
   ```
   9.83s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pack_combiners
   9.77s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pack_combiners
   9.68s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pack_combiners
   5.47s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_and_main_outputs
   5.22s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_fusion_break
   5.21s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_outputs
   5.19s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_unfusable_side_inputs_with_separation
   5.18s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_to_element_pardo
   5.16s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_batch_pardo_fusion_break
   5.16s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint_draining_sdf
   5.11s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_flatten_same_pcollections
   5.11s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_assert_that
   5.01s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_state_only
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_flattened_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_multimap_side_input
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_to_element_pardo
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_callbacks_with_exception
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_windowed_side_inputs
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_combine_per_key
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_crazy_draining_sdf
   5.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_pardo_overlapping_windows
   4.99s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_inputs
   4.94s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_custom_merging_window
   4.88s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_custom_merging_window
   4.86s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_outputs
   4.82s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_batch_rebatch_pardos
   4.68s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_state_timers_non_standard_coder
   4.67s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_windowing
   4.65s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_multimap_side_input_type_coercion
   4.64s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_and_main_outputs
   4.56s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint_sdf
   4.56s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_pardo_side_inputs
   4.51s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_pardo_side_and_main_outputs
   4.44s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_timers_clear
   4.42s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_multimap_multiside_input
   4.41s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_pardo_side_input_dependencies
   4.38s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf_synthetic_source
   4.25s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithDisabledCaching::test_register_finalizations
   4.18s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_flatten
   4.00s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_sdf_synthetic_source
   3.59s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpc::test_register_finalizations
   3.14s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_multimap_side_input_type_coercion
   2.88s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_reshuffle
   2.26s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_checkpoint
   2.07s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithGrpcAndMultiWorkers::test_sdf_with_dofn_as_restriction_provider
   1.97s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerSplitTest::test_split_crazy_sdf
   1.23s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithBundleRepeatAndMultiWorkers::test_pardo_side_and_main_outputs
   1.17s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTest::test_register_finalizations
   1.14s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithBundleRepeatAndMultiWorkers::test_pack_combiners
   0.93s call     apache_beam/runners/portability/fn_api_runner/fn_runner_test.py::FnApiRunnerTestWithMultiWorkers::test_pack_combiners
   
   ```


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kileys commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
kileys commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1215449843

   Yes, not a release blocker


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] pabloem commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1209724811

   I believe (though not sure) that this is due to fnapirunner closing grpc connections and should not be considered a perf regression


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205364956

   > 
   
   Yes, the test is waiting for state_server and sometimes control_server to shutdown. This did not occur in grpc<1.46.0.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] pabloem commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
pabloem commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1209724093

   i'lltake a look thanks


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] tvalentyn commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
tvalentyn commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205337048

   I think mocking the value in tests is fine if it does not result in additional flakiness. What happens if the server is shut down prematurely - will it restart or requests will just fail?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] aaltay commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
aaltay commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205910504

   Where is the perf regression tracked?


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] tvalentyn commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
tvalentyn commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205331685

   I was able to bisect grpc commits to debug another issue, while building grpc from sources, in case it helps, my notes are in https://github.com/apache/beam/issues/22533#issuecomment-1200062320


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] kileys commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
kileys commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1205792982

   Sorry, I meant that it may be a blocker. There is a performance regression as well


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


[GitHub] [beam] Abacn commented on issue #22283: [Bug]: Python Lots of fn runner test items cost exactly 5 seconds to run

Posted by GitBox <gi...@apache.org>.
Abacn commented on issue #22283:
URL: https://github.com/apache/beam/issues/22283#issuecomment-1248511772

   .take-issue


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@beam.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org