Hi to all,
I was trying to check whether our jobs are properly typed or not. I've started disabling generic types[1] in order to discover untyped transformations and so I added the proper returns() to operators. Unfortunately there are jobs where we serialize Thrift and DateTime objects, so I need to properly configure the serializers in the ExecutionEnvironment: env.registerTypeWithKryoSerializer(DateTime.class, JodaDateTimeSerializer.class); env.getConfig().addDefaultKryoSerializer(EntitonAtom.class, TBaseSerializer.class); env.getConfig().addDefaultKryoSerializer(EntitonQuad.class, TBaseSerializer.class); Those jobs don't work when I disable generic types and I get the following exception: Exception in thread "main" java.lang.UnsupportedOperationException: Generic types have been disabled in the ExecutionConfig and type xxx.EntitonAtom is treated as a generic type. I have a couple of questions:
Best, Flavio [1] env.getConfig().disableGenericTypes(); |
Hi, Flavio
You can try below and do not make disableGenericTypes and see what happens. env.registerTypeWithKryoSerializer(DateTime.class, JodaDateTimeSerializer.class); env.registerTypeWithKryoSerializer(EntitonAtom.class, TBaseSerializer.class); env.registerTypeWithKryoSerializer(EntitonQuad.class, TBaseSerializer.class); Cheers Minglei
|
Hi Minglei,
using the registerTypeWithKryoSerializer with the 3 classes works (without disableGenericTypes) but the problem is that I would like to avoid Kryo serialization if this is useful to speedup the job performance, and thus I'd like to be able to run all jobs with disableGenericTypes. Best, Flavio On Wed, Jul 18, 2018 at 11:10 AM, zhangminglei <[hidden email]> wrote:
|
Hi Flavio,
According to the current implementation of `disableGenericTypes`, the exception you get should be valid because Kryo still has to be used for `EntitonAtom` which might be classified as generic (non-serialisable by Flink). You cannot specify exceptions for this check at the moment. If you want to have control for which classes Kryo can be used and still activate `disableGenericTypes`, currently you can create your own type info where you provide your own Flink serialiser (extends TypeSerializer) which can internally use Kryo (KryoSerializer). You can do it using class annotation: This way Flink should not treat it as generic a type. Cheers, Andrey
|
Another way would be also to make `EntitonAtom` extend `org.apache.flink.types.Value` and implement `IOReadableWritable` using custom (Kryo) serialisation.
|
Free forum by Nabble | Edit this page |