Closed
Description
Search before asking
- I searched in the issues and found nothing similar.
Motivation
Q1:
When I am reading incremental streams, if I use the parameter "scan.timestamp-millis",
when the snapshot corresponding to the timestamp expires, an error will be reported.
I think there should be a bottom-back policy :
scan.timestamp-millis < snapshot earliest timestamp, then read from EARLIEST
scan.timestamp-millis > snapshot latest timestamp, then scans-mode = 'latest'
Q2:
The parallelism is greater than the number of buckets during stream read, and the excess slots are not used
Solution
No response
Anything else?
No response
Are you willing to submit a PR?
- I'm willing to submit a PR!
Metadata
Metadata
Assignees
Labels
Type
Projects
Milestone
Relationships
Development
No branches or pull requests
Activity
tsreaper commentedon Mar 27, 2023
Hi! Thanks for reporting this issue. However I'd like to discuss your questions a bit more.
In Q1 you mentioned that
What error is reported under what condition? Could you explain what you're doing in detail when the error occurs? Could you add a complete exception stack in the issue description?
You also mentioned that
As far as I remember, our current implementation has exactly the same effect. Is the actual behavior different? If yes, what is the actual behavior?
In Q2 you said that
This is expected for Flink sources. Users should either decrease the source parallelism or change bucket number to a larger value. Or are you suggesting something different? What would you like to suggest for this scenario?
wxplovecc commentedon Mar 27, 2023
Hi @tsreaper in Q1, user start streaming read give some fixed timestamp in sql, but if the the sql task restart then the error occurs
and in Q2 it`s expected in non partitioned table but for partiton table stream read should shuffle by partitons maybe
FangYongs commentedon Mar 27, 2023
Thanks @wxplovecc Which version are you using? Can you give the exception stack about Q1?
If I understand correctly, you submit a streaming job with paimon source which reads incremental data. If the source tasks of job restarted due to the failover, I think it will be restored with snapshot and splits from state instead of the fixed timestamp.
Of course, jobs may use memory state and source tasks will read from the fixed timestamp after restarted. But as @tsreaper mentioned above, the source subtask should read specific snapshot according to the timestamp from paimon without exception.
wxplovecc commentedon Mar 27, 2023
@FangYongs We are use the Master branch, you are right we restart the job without checkpoint . Do you think we should offer some strategy for this senior(restart without state) ?
FangYongs commentedon Mar 28, 2023
@wxplovecc What's the strategy do you mean for restarting without state? Could you describe it in detail? If a job has no state, I think it will calculate snapshot id according to the fixed timestamp without throwing an exception after restart
wxplovecc commentedon Mar 28, 2023
@FangYongs Maybe like this
scan.timestamp-millis
< snapshot earliest timestamp, then read from EARLIESTscan.timestamp-millis
> snapshot latest timestamp, then scans-mode = 'latest'when the fixed timestamp was illegality
or should we introduce consumer group like kafka, independent maintenance of consumption offsets
FangYongs commentedon Mar 28, 2023
As @tsreaper mentioned above, I think the current strategy of
scan.timestamp-millis
in paimon is just as you described:scan.timestamp-millis < snapshot earliest timestamp, then read from EARLIEST
scan.timestamp-millis > snapshot latest timestamp, then scans-mode = 'latest'
If it doesn't act as above, I think it's a bug
tsreaper commentedon Mar 28, 2023
Paimon Flink connector already supports this feature. If you set
sink.partition-shuffle
to true then the records will be shuffled both by bucket and by partition. See https://paimon.apache.org/docs/master/maintenance/configurations/#flinkconnectoroptions for more detail.@JingsongLi maybe we should remove this option and always shuffle by both bucket and partition. I don't see any disadvantage if we do so.
wxplovecc commentedon Mar 29, 2023
@FangYongs I think so. Not in genPlan phase. Below was the stack

The root case was the source task expire snapshot when stream read task startup
JingsongLi commentedon Apr 6, 2023
Sink has already supported it, but source dose not.
siyangzeng commentedon Aug 16, 2023
done