Ok, I can't make Option comparable, so the only way that I see is to
translate a key to a Comparable data structure and use it (as it was
alluded to in your example above). Thank you for clarification!

Thanks,
Timur

On Wed, Mar 30, 2016 at 9:22 AM, Chiwan Park <chiwanp...@apache.org> wrote:

> Hi Timur,
>
> Sorry for confusing. I meant KeySelector.
>
> `GenericType<T>` could be used as a key type if the `T` implements
> `Comparable`. For example, `GenericType<Integer>` could be used as a key
> type but `GenericType<scala.Tuple2>` could not.
>
> About my example in previous mail, the type of key is `Int` because the
> return type of KeySelector is `Int`. `TypeInformation<Int>` is not generic
> type.
>
> Regards,
> Chiwan Park
>
> > On Mar 31, 2016, at 1:09 AM, Timur Fayruzov <timur.fairu...@gmail.com>
> wrote:
> >
> > Thank you for your answers, Chiwan! That would mean that a generic type
> can't be used as a key in general? This is a non-obvious limitation of
> Flink DSL that I didn't see in documentation.
> >
> > Could you please elaborate what you mean by KeyExtractor? I see that
> inside `where` operator an instance of KeySelector is initialized, but I
> don't see how can I pass a custom KeySelector in.
> >
> > Thanks,
> > Timur
> >
> > On Wed, Mar 30, 2016 at 12:53 AM, Chiwan Park <chiwanp...@apache.org>
> wrote:
> > Hi Timur,
> >
> > Because Option[T] is not comparable type generally (if T is a POJO
> type), you cannot use Option[T] as a key type. I think you have to
> implement KeyExtractor to compare objects including Option[T]s.
> >
> > ```
> > case class MyKey(k1: Option[String], k2: Option[String])
> >
> > val data1 = env.fromElements(MyKey(Some("a"), None), MyKey(Some("a"),
> Some("c")))
> > val data2 = env.fromElements(MyKey(Some("b"), None), MyKey(Some("a"),
> Some("c")))
> >
> > data1.join(data2)
> >   .where(_.hashCode())
> >   .equalTo(_.hashCode()).apply {
> >     (left: MyKey, right: MyKey) => (left, right)
> >   }.print()
> > ```
> >
> > Note that the approach in example (using hashCode()) cannot be applied
> to sort task.
> >
> > Regards,
> > Chiwan Park
> >
> > > On Mar 30, 2016, at 2:37 AM, Timur Fayruzov <timur.fairu...@gmail.com>
> wrote:
> > >
> > > There is some more detail to this question that I missed initially. It
> turns out that my key is a case class of a form MyKey(k1: Option[String],
> k2: Option[String]). Keys.SelectorFunctionKeys is performing a recursive
> check whether every element of the MyKey class can be a key and fails when
> encountering an Option.
> > >
> > > Is it possible to work around this situation without giving up
> Options? Inability to use Options in Domain objects could be really
> frustrating.
> > >
> > > Thanks,
> > > Timur
> > >
> > > On Tue, Mar 29, 2016 at 10:19 AM, Timur Fayruzov <
> timur.fairu...@gmail.com> wrote:
> > > Hello,
> > >
> > > I'm evaluating Flink and one thing I noticed is Option[A] can't be
> used as a key for coGroup (looking specifically here:
> https://github.com/apache/flink/blob/master/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/OptionTypeInfo.scala#L39).
> I'm not clear about the reason of this and appreciate if someone can
> explain.
> > >
> > > Thanks,
> > > Timur
> > >
> >
> >
>
>

Reply via email to