[
https://issues.apache.org/jira/browse/FLINK-2809?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=14947027#comment-14947027
]
ASF GitHub Bot commented on FLINK-2809:
---------------------------------------
Github user ggevay commented on a diff in the pull request:
https://github.com/apache/flink/pull/1217#discussion_r41404589
--- Diff:
flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/UnitSerializer.scala
---
@@ -0,0 +1,58 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements. See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.flink.api.scala.typeutils
+
+import org.apache.flink.api.common.typeutils.base.TypeSerializerSingleton
+import org.apache.flink.core.memory.{DataInputView, DataOutputView}
+
+class UnitSerializer extends TypeSerializerSingleton[Unit] {
+
+ def isImmutableType: Boolean = true
+
+ def createInstance: Unit = ()
+
+ def copy(from: Unit): Unit = ()
+
+ def copy(from: Unit, reuse: Unit): Unit = ()
+
+ def getLength: Int = 1
+
+ def serialize(record: Unit, target: DataOutputView) {
+ target.write(0)
+ }
+
+ def deserialize(source: DataInputView): Unit = {
+ source.readByte()
+ ()
+ }
+
+ def deserialize(reuse: Unit, source: DataInputView): Unit = {
+ source.readByte()
+ ()
+ }
+
+ def copy(source: DataInputView, target: DataOutputView) {
+ target.write(source.readByte)
+ }
+
+ override def hashCode(): Int = classOf[UnitSerializer].hashCode
+
+ override def canEqual(obj: scala.Any): Boolean = {
+ obj.isInstanceOf[Unit]
--- End diff --
Sorry, I have now changed it to ```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)