No I'm using the maven builds, I could try a nightly if you like.

On Thu, Jan 29, 2015 at 4:34 PM, Aljoscha Krettek <[email protected]>
wrote:

> Hi Arvid,
> this indeed looks like a ClassLoader problem. Are you running a
> version that you compiled yourself? If yes, could you try replacing
> line 49 in RuntimeStatefulSerializerFactory.java with this line:
>
> this.loader = Thread.currentThread().getContextClassLoader();
>
> Cheers,
> Aljoscha
>
> On Thu, Jan 29, 2015 at 4:25 PM, Alexander Alexandrov
> <[email protected]> wrote:
> > Forget what I just said, didn't realize that it's Scala :)
> >
> > 2015-01-29 16:24 GMT+01:00 Alexander Alexandrov <
> > [email protected]>:
> >
> >> have you tried declaring your UDF classes (e.g. TotalRankDistribution)
> as
> >> static?
> >>
> >> 2015-01-29 16:14 GMT+01:00 Arvid Heise <[email protected]>:
> >>
> >>> Hi Flinker,
> >>>
> >>> I'm currently desparetely trying to get a workflow to run remotely on a
> >>> server. The workflow works fine in the local execution environment
> (both
> >>> with ExecutionEnvironment.getExecutionEnvironment and
> >>> ExecutionEnvironment.createLocalEnvironment(2)).
> >>>
> >>> On the server, I get
> >>>
> >>> 01/29/2015 16:05:22:    GroupReduce (GroupReduce at
> >>>
> de.bund.bfr.flink.outbreakanalysis.OutbreakAnalysis$.main(OutbreakAnalysis.scala:79))
> >>> (1/1) switched to FAILED
> >>> java.lang.RuntimeException: : Repeated instantiation of serializer
> failed.
> >>>         at
> >>>
> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:340)
> >>>         at
> >>>
> org.apache.flink.runtime.execution.RuntimeEnvironment.run(RuntimeEnvironment.java:257)
> >>>         at java.lang.Thread.run(Thread.java:722)
> >>> Caused by: java.lang.RuntimeException: Repeated instantiation of
> >>> serializer failed.
> >>>         at
> >>>
> org.apache.flink.api.java.typeutils.runtime.RuntimeStatefulSerializerFactory.getSerializer(RuntimeStatefulSerializerFactory.java:102)
> >>>         at
> >>>
> org.apache.flink.api.java.typeutils.runtime.TupleComparatorBase.instantiateDeserializationUtils(TupleComparatorBase.java:267)
> >>>         at
> >>>
> org.apache.flink.api.scala.typeutils.CaseClassComparator.duplicate(CaseClassComparator.scala:42)
> >>>         at
> >>>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.<init>(UnilateralSortMerger.java:360)
> >>>         at
> >>>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.<init>(UnilateralSortMerger.java:227)
> >>>         at
> >>>
> org.apache.flink.runtime.operators.sort.UnilateralSortMerger.<init>(UnilateralSortMerger.java:195)
> >>>         at
> >>>
> org.apache.flink.runtime.operators.RegularPactTask.initInputLocalStrategy(RegularPactTask.java:981)
> >>>         at
> >>>
> org.apache.flink.runtime.operators.RegularPactTask.initLocalStrategies(RegularPactTask.java:842)
> >>>         at
> >>>
> org.apache.flink.runtime.operators.RegularPactTask.invoke(RegularPactTask.java:336)
> >>>         ... 2 more
> >>> Caused by: java.lang.ClassNotFoundException:
> >>>
> de.bund.bfr.flink.outbreakanalysis.OutbreakAnalysis$$anon$13$$anon$14$$anon$15
> >>>
> >>> The class is of course in the submitted jar and I use the same flink
> and
> >>> java version on both machines. Could it be that the wrong class loader
> is
> >>> used for duplicating the comparator?
> >>>
> >>> The code is available here
> >>>
> https://github.com/SiLeBAT/Other/tree/master/de.bund.bfr.flink.outbreakanalysis
> >>> .
> >>> If you need help to invoke the program, please msg me privately, so
> that
> >>> I can send you the non-public datasets.
> >>>
> >>> Thank you in advance,
> >>>
> >>> Arvid
> >>>
> >>
> >>
>

Reply via email to