It would still work ________________________________ 发件人: Richard Cheung <rcheungsi...@gmail.com> 发送时间: 星期二, 五月 20, 2025 4:08:00 上午 收件人: Zhanghao Chen <zhanghao.c...@outlook.com> 抄送: Мосин Николай <mosin...@yandex.ru>; Schwalbe Matthias <matthias.schwa...@viseca.ch>; user@flink.apache.org <user@flink.apache.org> 主题: Re: Apache Flink Serialization Question
Hi all, Thanks again for the help! I have one more follow up question regarding Flink and serialization on v1.18. I know state schema evolution is supported for POJOs in Flink. However, if my class uses the POJO serializer but has a field that falls back to Kryo (such as UUID), would it still be eligible for Flink’s state schema evolution support? Best, Richard On Fri, May 16, 2025 at 3:59 AM Zhanghao Chen <zhanghao.c...@outlook.com<mailto:zhanghao.c...@outlook.com>> wrote: Flink 2.0 will work. You may use Types.LIST for lists and Types.MAP for sets (mocked by a Map) for that. Notice that Flink's built-in LIST does not support null element and MAP type does not support null key, and neither support null collection. In Flink 2.0, we've added special treatment of these nullable cases. Best, Zhanghao Chen ________________________________ From: Мосин Николай <mosin...@yandex.ru<mailto:mosin...@yandex.ru>> Sent: Friday, May 16, 2025 0:02 To: Richard Cheung <rcheungsi...@gmail.com<mailto:rcheungsi...@gmail.com>> Cc: Schwalbe Matthias <matthias.schwa...@viseca.ch<mailto:matthias.schwa...@viseca.ch>>; Zhanghao Chen <zhanghao.c...@outlook.com<mailto:zhanghao.c...@outlook.com>>; user@flink.apache.org<mailto:user@flink.apache.org> <user@flink.apache.org<mailto:user@flink.apache.org>> Subject: Re: Apache Flink Serialization Question For List I just setup TypeInfoFactory like: public class SomeDataDTOTypeInfoFactory extends TypeInfoFactory<SomeDataDTO> { @Override public TypeInformation<SomeDataDTO> createTypeInfo(Type t, Map<String, TypeInformation<?>> genericParameters) { return Types.POJO(SomeDataDTO.class, new HashMap<String, TypeInformation<?>>() { { .... put("sessionId", UUIDTypeInfo.INSTANCE); put("tags", Types.LIST(Types.STRING)); .... } }); } } and annotate DTO @TypeInfo(SomeDataDTOTypeInfoFactory.class) public class SomeDataDTO { .... private UUID sessionId; private List<String> tags = new ArrayList<>(); .... But for Set I don't found workaround and as I understand it must be replaced by List ---------------- Кому: Mosin Nick (mosin...@yandex.ru<mailto:mosin...@yandex.ru>); Копия: Schwalbe Matthias (matthias.schwa...@viseca.ch<mailto:matthias.schwa...@viseca.ch>), Zhanghao Chen (zhanghao.c...@outlook.com<mailto:zhanghao.c...@outlook.com>), user@flink.apache.org<mailto:user@flink.apache.org>; Тема: Apache Flink Serialization Question; 15.05.2025, 18:27, "Richard Cheung" <rcheungsi...@gmail.com<mailto:rcheungsi...@gmail.com>>: Thanks for all the replies! I’ve decided to just update my UUID field to a String for POJO compliance. However, I’m getting the same log issues for List<String> and Set<String> saying that fields will be processed as GenericType. I want everything to be fully POJO compatible so I can have schema evolution for the future. Is there a workaround for this for POJO compliance in Flink v1.8 or would I have to upgrade to Flink v2 which supports common collection types for serialization or maybe the even upgrading to v2 won’t work? Best regards, Richard On Thu, May 15, 2025 at 9:06 AM Mosin Nick <mosin...@yandex.ru<mailto:mosin...@yandex.ru>> wrote: Github already contain some serializer for UUID https://github.com/gAmUssA/datalorean/blob/main/src/main/java/com/example/datadelorean/serialization/UUIDSerializer.java Work well for me ---------------- Кому: Zhanghao Chen (zhanghao.c...@outlook.com<mailto:zhanghao.c...@outlook.com>), Richard Cheung (rcheungsi...@gmail.com<mailto:rcheungsi...@gmail.com>), user@flink.apache.org<mailto:user@flink.apache.org> (user@flink.apache.org<mailto:user@flink.apache.org>); Тема: Apache Flink Serialization Question; 15.05.2025, 15:56, "Schwalbe Matthias" <matthias.schwa...@viseca.ch<mailto:matthias.schwa...@viseca.ch>>: Hi Richard, Same problem, 12 Flink versions later, I created my own TypeInformation/Serializer/Snapshot for UUID (Scala in that case), along: class UUIDTypeInformation extends TypeInformation[UUID] … class UUIDSerializer extends TupleSerializerBase[UUID]( … class UUIDSerializerSnapshot(serializer: UUIDSerializer) extends CompositeTypeSerializerSnapshot[UUID, UUIDSerializer](serializer) { … Altogether 100 lines of code, implementation is simple, you only need to implement the respective apis and ‘follow the types’ And the register the UUIDTypeInformation with the Flink type system. … unfortunately I haven’t got the permission to share the code … Hope that helps 😊 This From: Zhanghao Chen <zhanghao.c...@outlook.com<mailto:zhanghao.c...@outlook.com>> Sent: Wednesday, May 14, 2025 3:00 AM To: Richard Cheung <rcheungsi...@gmail.com<mailto:rcheungsi...@gmail.com>>; user@flink.apache.org<mailto:user@flink.apache.org> Subject: [External] Re: Apache Flink Serialization Question ⚠EXTERNAL MESSAGE – CAUTION: Think Before You Click ⚠ Flink still use PojoSerializer for the class while only using Kryo for the UUID field. Best, Zhanghao Chen ________________________________ From: Richard Cheung <rcheungsi...@gmail.com<mailto:rcheungsi...@gmail.com>> Sent: Wednesday, May 14, 2025 3:21 To: user@flink.apache.org<mailto:user@flink.apache.org> <user@flink.apache.org<mailto:user@flink.apache.org>> Subject: Apache Flink Serialization Question Hi all! I have a question about serialization of POJO data classes in Apache Flink v1.8 (Java). Here's some context: One of my data classes was initially not POJO compatible as it had final fields and no no-arg constructor. I removed the final modifiers and added a no-arg constructor, and confirmed that the serializer switched from KryoSerializer to PojoSerializer using TypeInformation and TypeSerializer to log the info. However, one of the fields in the class is a java.util.UUID, which is not POJO compatible. I see a log message saying that the UUID field cannot be used as a POJO type. The logs also say that the UUID field will be processed as a generic type which means that, to my knowledge, the KryoSerializer will be used. These logs appear after the one indicating that the PojoSerializer is being used for my data class. My question is: Does the presence of the UUID field cause the entire class to be serialized with KryoSerializer or does Flink still use PojoSerializer for the class while only using Kryo for the UUID field? Best regards, Richard Diese Nachricht ist ausschliesslich für den Adressaten bestimmt und beinhaltet unter Umständen vertrauliche Mitteilungen. Da die Vertraulichkeit von e-Mail-Nachrichten nicht gewährleistet werden kann, übernehmen wir keine Haftung für die Gewährung der Vertraulichkeit und Unversehrtheit dieser Mitteilung. Bei irrtümlicher Zustellung bitten wir Sie um Benachrichtigung per e-Mail und um Löschung dieser Nachricht sowie eventueller Anhänge. Jegliche unberechtigte Verwendung oder Verbreitung dieser Informationen ist streng verboten. This message is intended only for the named recipient and may contain confidential or privileged information. As the confidentiality of email communication cannot be guaranteed, we do not accept any responsibility for the confidentiality and the intactness of this message. If you have received it in error, please advise the sender by return e-mail and delete this message and any attachments. Any unauthorised use or dissemination of this information is strictly prohibited.