Hi Kenn,

unfortunately the support for this annotation is not as good as it could be. AFAIK it is currently supported only on Java Direct, Flink, Spark and DataFlow batch runners. DataFlow streaming does not support this. There was some discussion that the expansion could be implemented by a different type of "expansion", where a DoFn would be actually replaced with a different DoFn (containing the sorting state and timer), but this is not yet implement. I have a PoC implementation of this approach (not related to time-sorting, but a different type of expansion) that I would like to share soon.

 Jan

On 10/1/24 17:15, Kenneth Knowles wrote:
Also worth calling out RequiresTimeSortedInput (https://beam.apache.org/releases/javadoc/2.59.0/index.html?org/apache/beam/sdk/transforms/DoFn.RequiresTimeSortedInput.html).

It only operates at the level of a single stateful ParDo but this ordering will persist until the next shuffle on most runners (after a shuffle you'd need another sort). As with all stateful ParDos, the single-logical-threaded processing is only enforced per key+window so you can still achieve scale despite the ordering constraint.

I am less familiar with the Python SDK. I briefly looked but it may not be implemented there. (it is just an annotation that is passed on to the runner so it would be very easy to remedy this)

Kenn

On Mon, Sep 30, 2024 at 8:37 AM Danny McCormick via dev <dev@beam.apache.org> wrote:

    I'm not sure if I fully understand the use case. When you require
    ordering, do you need a set of transforms completed on all data
    before moving to the next set of transforms? Or do you need
    transforms to complete on a subset of the data before moving to
    the next subset of the data for the same transforms?

    If it is the former, you could consider using the Wait transform
    
<https://beam.apache.org/releases/javadoc/current/org/apache/beam/sdk/transforms/Wait.html>.
    If it's the latter, you probably need something like
    OrderedListState as XQ described above. Alternatively, if your
    data is partitioned in a way which is known ahead of time, you
    could transform the latter problem into the former problem by
    defining a transform per subset of data.

    Thanks,
    Danny

    On Sat, Sep 28, 2024 at 1:18 PM XQ Hu via dev
    <dev@beam.apache.org> wrote:

        Not exactly sure what your use case is. This year, at our Beam
        Summit, Shunping talked about Beam State and
        OrderedListStates:
        https://beamsummit.org/sessions/2024/introducing-ordered-list-states/.
        This might be helpful for you.

        On Sat, Sep 28, 2024 at 10:30 AM Settara Pramod
        <pramodsettar...@gmail.com> wrote:

            Hi Apache Beam Dev Team,

            First of all, thank you for developing such an amazing
            project and making it open-source.

            I have a use case where I encountered some limitations in
            using Apache Beam to solve my problem. I am working with
            workloads that are tied to specific sequences. My goal is
            to process workloads in order, where workloads with the
            same sequence should run concurrently, but workloads in
            the next sequence must wait until the previous sequence
            has fully completed.

            With Apache Beam, I found it difficult to achieve this
            balance. I was only able to process workloads either
            sequentially by enforcing strict ordering or run all
            workloads in parallel. Neither approach fully met my needs.

            I considered a workaround where one pipeline calls
            another, but I believe this would be inefficient and
            resource-intensive, leading to poor design and scalability.

            Is there a way to extend or customize Apache Beam to
            handle this use case more effectively? I would greatly
            appreciate any guidance or suggestions on this.

            I would also be happy to set up a call if further
            clarification is needed regarding my use case.

            Thanks & Regards
            Pramod

Reply via email to