[
https://issues.apache.org/jira/browse/FLINK-2809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14941359#comment-14941359
]
ASF GitHub Bot commented on FLINK-2809:
---------------------------------------
GitHub user ggevay opened a pull request:
https://github.com/apache/flink/pull/1217
[FLINK-2809] [scala-api] Added UnitTypeInfo and UnitSerializer.
Created UnitTypeInfo and UnitSerializer, which will be created for a
DataSet[Unit].
Also added a test.
There is a funny situation in CollectionInputFormat.checkCollection: when
ExecutionEnvironment.fromCollection calls it, the call to type.getTypeClass()
returns void.class, even though it should be classOf[Unit]. This is probably
some automatic conversion that happens when classOf[Unit] passes from the Scala
world to the Java world. I worked around this by adding a check for this
specific case.
You can merge this pull request into a Git repository by running:
$ git pull https://github.com/ggevay/flink unitTypeInfo
Alternatively you can review and apply these changes as the patch at:
https://github.com/apache/flink/pull/1217.patch
To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:
This closes #1217
----
commit 0fe37101586cd01148fe4bb629f8ad743a32778d
Author: Gabor Gevay <[email protected]>
Date: 2015-10-02T16:04:59Z
[FLINK-2809] [scala-api] Added UnitTypeInfo and UnitSerializer.
----
> DataSet[Unit] doesn't work
> --------------------------
>
> Key: FLINK-2809
> URL: https://issues.apache.org/jira/browse/FLINK-2809
> Project: Flink
> Issue Type: Bug
> Components: Scala API
> Reporter: Gabor Gevay
> Assignee: Gabor Gevay
> Priority: Minor
>
> The following code creates a DataSet\[Unit\]:
> val env = ExecutionEnvironment.createLocalEnvironment()
> val a = env.fromElements(1,2,3)
> val b = a.map (_ => ())
> b.writeAsText("/tmp/xxx")
> env.execute()
> This doesn't work, because a VoidSerializer is created, which can't cope with
> a BoxedUnit. See exception below.
> I'm now thinking about creating a UnitSerializer class.
> org.apache.flink.runtime.client.JobExecutionException: Job execution failed.
> at
> org.apache.flink.runtime.jobmanager.JobManager$$anonfun$receiveWithLogMessages$1.applyOrElse(JobManager.scala:314)
> at
> scala.runtime.AbstractPartialFunction.apply(AbstractPartialFunction.scala:36)
> at
> org.apache.flink.runtime.ActorLogMessages$$anon$1.apply(ActorLogMessages.scala:36)
> at
> org.apache.flink.runtime.ActorLogMessages$$anon$1.apply(ActorLogMessages.scala:29)
> at scala.PartialFunction$class.applyOrElse(PartialFunction.scala:123)
> at
> org.apache.flink.runtime.ActorLogMessages$$anon$1.applyOrElse(ActorLogMessages.scala:29)
> at akka.actor.Actor$class.aroundReceive(Actor.scala:465)
> at
> org.apache.flink.runtime.jobmanager.JobManager.aroundReceive(JobManager.scala:92)
> at akka.actor.ActorCell.receiveMessage(ActorCell.scala:516)
> at akka.actor.ActorCell.invoke(ActorCell.scala:487)
> at akka.dispatch.Mailbox.processMailbox(Mailbox.scala:254)
> at akka.dispatch.Mailbox.run(Mailbox.scala:221)
> at akka.dispatch.Mailbox.exec(Mailbox.scala:231)
> at scala.concurrent.forkjoin.ForkJoinTask.doExec(ForkJoinTask.java:260)
> at
> scala.concurrent.forkjoin.ForkJoinPool$WorkQueue.runTask(ForkJoinPool.java:1339)
> at
> scala.concurrent.forkjoin.ForkJoinPool.runWorker(ForkJoinPool.java:1979)
> at
> scala.concurrent.forkjoin.ForkJoinWorkerThread.run(ForkJoinWorkerThread.java:107)
> Caused by: java.lang.ClassCastException: scala.runtime.BoxedUnit cannot be
> cast to java.lang.Void
> at
> org.apache.flink.api.common.typeutils.base.VoidSerializer.serialize(VoidSerializer.java:26)
> at
> org.apache.flink.runtime.plugable.SerializationDelegate.write(SerializationDelegate.java:51)
> at
> org.apache.flink.runtime.io.network.api.serialization.SpanningRecordSerializer.addRecord(SpanningRecordSerializer.java:76)
> at
> org.apache.flink.runtime.io.network.api.writer.RecordWriter.emit(RecordWriter.java:83)
> at
> org.apache.flink.runtime.operators.shipping.OutputCollector.collect(OutputCollector.java:65)
> at
> org.apache.flink.runtime.operators.chaining.ChainedMapDriver.collect(ChainedMapDriver.java:78)
> at
> org.apache.flink.runtime.operators.DataSourceTask.invoke(DataSourceTask.java:177)
> at org.apache.flink.runtime.taskmanager.Task.run(Task.java:564)
> at java.lang.Thread.run(Thread.java:745)
--
This message was sent by Atlassian JIRA
(v6.3.4#6332)