It was planned for 1.12 but didn't make it. 1.13 should fully implement
FLIP-136. I just created issues to monitor the progress:
https://issues.apache.org/jira/browse/FLINK-19976
Regards,
Timo
On 04.11.20 18:43, Rex Fenley wrote:
Thank you for the info!
Is there a timetable for when the next
Thank you for the info!
Is there a timetable for when the next version with this change might
release?
On Wed, Nov 4, 2020 at 2:44 AM Timo Walther wrote:
> Hi Rex,
>
> sorry for the late reply. POJOs will have much better support in the
> upcoming Flink versions because they have been fully int
Hi Rex,
sorry for the late reply. POJOs will have much better support in the
upcoming Flink versions because they have been fully integrated with the
new table type system mentioned in FLIP-37 [1] (e.g. support for
immutable POJOs and nested DataTypeHints etc).
For queries, scalar, and table
My jobs normally use the blink planner, I noticed with this test that may
not be the case.
On Mon, Nov 2, 2020 at 12:38 PM Rex Fenley wrote:
> Flink 1.11.2 with Scala 2.12
>
> Error:
> [info] JobScalaTest:
> [info] - dummy *** FAILED ***
> [info] org.apache.flink.table.api.ValidationException:
Flink 1.11.2 with Scala 2.12
Error:
[info] JobScalaTest:
[info] - dummy *** FAILED ***
[info] org.apache.flink.table.api.ValidationException: Field types of
query result and registered TableSink do not match.
[info] Query schema: [user: BIGINT, product: ROW<`name`
VARCHAR(2147483647), `id` BIGI
@Timo: Is this sth that would work when using the new type stack? From
the message I'm assuming it's using the older type stack.
@Rex: Which Flink version are you using and could you maybe post the
code snipped that you use to do conversions?
Best,
Aljoscha
On 02.11.20 06:50, Rex Fenley wrot
Maybe this is related to this issue?
https://issues.apache.org/jira/browse/FLINK-17683
On Fri, Oct 30, 2020 at 8:43 PM Rex Fenley wrote:
> Correction, I'm using Scala case classes not strictly Java POJOs just to
> be clear.
>
> On Fri, Oct 30, 2020 at 7:56 PM Rex Fenley wrote:
>
>> Hello,
>>
>>
Correction, I'm using Scala case classes not strictly Java POJOs just to be
clear.
On Fri, Oct 30, 2020 at 7:56 PM Rex Fenley wrote:
> Hello,
>
> I keep running into trouble moving between DataStream and SQL with POJOs
> because my nested POJOs turn into LEGACY('STRUCTURED_TYPE', is there any
>