[
https://issues.apache.org/jira/browse/FLINK-26882?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17514562#comment-17514562
]
Yun Tang commented on FLINK-26882:
----------------------------------
Since my local CI
https://dev.azure.com/myasuka/flink/_build/results?buildId=415&view=results had
turned green. I have already merged the fix.
* The reason why we create another *RescaleCheckpointManuallyITCase* is due to
*RescalingITCase* only targets for savepoint rescale. This has some historical
reason for the mechanism of Flink rescale. If you take a look at current Flink
official document about [checkpoints difference to
savepoints|https://nightlies.apache.org/flink/flink-docs-release-1.15/docs/ops/state/checkpoints/#difference-to-savepoints]
: {{"Checkpoints have a few differences from savepoints. They do not support
Flink specific features like rescaling."}}
>From my point of view, this means Flink community wants to hold some space to
>explain and does not promise checkpoint could support rescale at any time.
>That's why {{RescalingITCase}} only focus on savepoint rescale. On the other
>hand, RocksDB incremental checkpoint is the factual choice for many companies
>in production environment, and they heavily depend on the incremenatl
>checkpoint rescaling feature. In FLINK-21321, we tried to improve the RocksDB
>rescale performance. Since we did not want to touch the existing scope of
>{{RescalingITCase}}, not to mentition that case has some other unrelated test
>parameters, we then introduce another IT case targets for RocksDB rescale due
>to we changed the RocksDB rescale implementation.
* For current RestoreUpgradedJobITCase, I feel a bit weird about that case as
it just used operator state, which means there is no difference with native
savepoint and canonical savepoint.
[~akalashnikov] I noticed that your PR of FLINK-26134 had dropped the limit of
checkpoint rescaling, and I think it's okay to unify them together in one IT
case if we follow the direction of FLIP-203. However, that test must include
verifing on keyed state and all kinds of state-backends.
> Unaligned checkpoint with 0s timeout would fail
> RescaleCheckpointManuallyITCase
> -------------------------------------------------------------------------------
>
> Key: FLINK-26882
> URL: https://issues.apache.org/jira/browse/FLINK-26882
> Project: Flink
> Issue Type: Bug
> Components: Runtime / Checkpointing, Tests
> Affects Versions: 1.16.0
> Reporter: Yun Tang
> Assignee: Yun Tang
> Priority: Blocker
> Labels: pull-request-available
> Fix For: 1.16.0
>
>
> Once we make {{execution.checkpointing.unaligned: true}} and
> {{execution.checkpointing.alignment-timeout: PT0S}}, the
> RescaleCheckpointManuallyITCase.testCheckpointRescalingInKeyedState would
> fail then.
> Borken instances:
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33776&view=logs&j=5c8e7682-d68f-54d1-16a2-a09310218a49&t=86f654fa-ab48-5c1a-25f4-7e7f6afb9bba&l=5623
>
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33787&view=logs&j=5c8e7682-d68f-54d1-16a2-a09310218a49&t=86f654fa-ab48-5c1a-25f4-7e7f6afb9bba&l=5626
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33787&view=logs&j=a57e0635-3fad-5b08-57c7-a4142d7d6fa9&t=2ef0effc-1da1-50e5-c2bd-aab434b1c5b7&l=12409
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=5c8e7682-d68f-54d1-16a2-a09310218a49&t=86f654fa-ab48-5c1a-25f4-7e7f6afb9bba&l=5629
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=a57e0635-3fad-5b08-57c7-a4142d7d6fa9&t=2ef0effc-1da1-50e5-c2bd-aab434b1c5b7&l=12409
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=baf26b34-3c6a-54e8-f93f-cf269b32f802&t=8c9d126d-57d2-5a9e-a8c8-ff53f7b35cd9&l=5733
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=a549b384-c55a-52c0-c451-00e0477ab6db&t=eef5922c-08d9-5ba3-7299-8393476594e7&l=12575
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=2c3cbe13-dee0-5837-cf47-3053da9a8a78&t=b78d9d30-509a-5cea-1fef-db7abaa325ae&l=5838
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=b0a398c0-685b-599c-eb57-c8c2a771138e&t=747432ad-a576-5911-1e2a-68c6bedc248a&l=12931
> https://dev.azure.com/apache-flink/apache-flink/_build/results?buildId=33779&view=logs&j=8fd9202e-fd17-5b26-353c-ac1ff76c8f28&t=ea7cf968-e585-52cb-e0fc-f48de023a7ca&l=5682
--
This message was sent by Atlassian Jira
(v8.20.1#820001)