The sorting by bytes is a deliberate limitation of this particular
approach. It basically assumes you are using bytes-based shuffle under the
hood, so invoking a language-specific comparator would be something new. I
know +Ben had some ideas about this.

Kenn

On Wed, May 30, 2018 at 8:53 AM David Morávek <david.mora...@gmail.com>
wrote:

> Thanks for pointing us the right direction. We'll try to prototype custom
> translation for Spark runner within next sprint. In order to do so, I have
> few questions:
>
> 1) Should we move SortValues tranform to beam-sdks-java-core or just add
> it as spark runner dependency?
> 2) I think we should try to make SortValues more flexible by letting user
> to provide custom value comparator, sorting lexicographically by secondary
> key may be painful in some use cases. What do you think?
>
> side note:
> I agree that usually top n values, that fit in memory are sufficient and
> we can combine them using PQ, but in practice we still have pipelines that
> need to do top N selection over data that do not fit in memory for a single
> key.
>
> D.
>
> On Wed, May 30, 2018 at 5:28 PM, Lukasz Cwik <lc...@google.com> wrote:
>
>> SortValues does not have a defined & documented URN yet. Once a Runner is
>> providing such an override, it will happen. No runner publicly provides one
>> to my knowledge.
>>
>> On Wed, May 30, 2018 at 8:08 AM Kenneth Knowles <k...@google.com> wrote:
>>
>>> I can see a few usability issues here. Totally agree w/ Luke, just
>>> noting:
>>>
>>>  - The naming is slightly misleading because SortValues is actually
>>> already GBK+SortValues.
>>>  - It also makes things look less supported when they are in the
>>> extensions/ folder. I'd say we should have a better place to put such a
>>> library if it is the official public implementation. The word "extensions"
>>> doesn't seem particularly accurate or meaningful to me.
>>>
>>> Q: Does SortValues have a defined & documented URN yet?
>>>
>>> Kenn
>>>
>>> On Wed, May 30, 2018 at 7:52 AM Lukasz Cwik <lc...@google.com> wrote:
>>>
>>>> Each runner can choose to override the SortValues PTransform with their
>>>> own internal offering. For example Spark overrides global combine[1] during
>>>> pipeline translation. If Spark detected the SortValues PTransform during
>>>> translation, it could override the offering with something that used
>>>> repartitionAndSortWithinPartitions.
>>>>
>>>> GroupByKeyAndSortValuesOnly inside Dataflow exists to support a
>>>> specific use case. Users should rely on SortValues as it is the public
>>>> implementation for sorting.
>>>>
>>>> 1:
>>>> https://github.com/apache/beam/blob/85dcab56268fbac923ffd5885489ee154f097fc5/runners/spark/src/main/java/org/apache/beam/runners/spark/translation/TransformTranslator.java#L200
>>>>
>>>> As a side note, its uncommon where you need to sort all values, usually
>>>> top 100 suffices and can be implemented much more efficiently with a
>>>> combiner when compared to sorting.
>>>>
>>>> On Wed, May 30, 2018 at 3:38 AM <marek-simu...@seznam.cz> wrote:
>>>>
>>>>> Hi,
>>>>>  I have question I am trying to do translation in dsl-euphoria for
>>>>> “GroupByKey with sorted values within key” to Beam. I am aware of java sdk
>>>>> extensions SortValues, but it doesn’t have sufficient abstraction for
>>>>> runners.
>>>>>
>>>>> I noticed that in DataflowRunner there is translation of batch
>>>>> GroupByKey to GroupByKeyAndSortValuesOnly but is it considered to have it
>>>>> in beam core so for example SparkRunner could translate “GroupByKey with
>>>>> sorted values within key” with their internals such as
>>>>> repartitionAndSortWithinPartitions.
>>>>>
>>>>> Thank you.
>>>>> Marek Simunek
>>>>>
>>>>
>

Reply via email to