You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by ch...@apache.org on 2015/10/07 18:42:12 UTC

flink git commit: [FLINK-2806] [scala-api] Add a TypeInformation[Nothing]

Repository: flink
Updated Branches:
  refs/heads/master 8d62033c2 -> 6e42f9fc0


[FLINK-2806] [scala-api] Add a TypeInformation[Nothing]

This closes #1212.


Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/6e42f9fc
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/6e42f9fc
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/6e42f9fc

Branch: refs/heads/master
Commit: 6e42f9fc0804886efd57994a72feb226e26958cc
Parents: 8d62033
Author: Gabor Gevay <gg...@gmail.com>
Authored: Fri Oct 2 10:14:01 2015 +0200
Committer: Chiwan Park <ch...@apache.org>
Committed: Wed Oct 7 18:40:19 2015 +0200

----------------------------------------------------------------------
 .../api/scala/codegen/TypeInformationGen.scala  |  5 ++-
 .../org/apache/flink/api/scala/package.scala    |  7 ++-
 .../scala/typeutils/ScalaNothingTypeInfo.scala  | 47 ++++++++++++++++++++
 .../scala/types/TypeInformationGenTest.scala    | 16 +++++++
 4 files changed, 72 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/flink/blob/6e42f9fc/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
index aa76fcc..499bc5b 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/codegen/TypeInformationGen.scala
@@ -23,7 +23,7 @@ import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo._
 import org.apache.flink.api.common.typeutils._
 import org.apache.flink.api.java.typeutils._
-import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo, ScalaNothingTypeInfo}
 import org.apache.flink.types.Value
 import org.apache.hadoop.io.Writable
 
@@ -59,7 +59,8 @@ private[flink] trait TypeInformationGen[C <: Context] {
     case p : PrimitiveDescriptor => mkPrimitiveTypeInfo(p.tpe)
     case p : BoxedPrimitiveDescriptor => mkPrimitiveTypeInfo(p.tpe)
 
-    case n: NothingDesciptor => reify { null.asInstanceOf[TypeInformation[T]] }
+    case n: NothingDesciptor =>
+      reify { new ScalaNothingTypeInfo().asInstanceOf[TypeInformation[T]] }
 
     case e: EitherDescriptor => mkEitherTypeInfo(e)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6e42f9fc/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
index db9c68c..c006888 100644
--- a/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
@@ -22,7 +22,7 @@ import org.apache.flink.api.common.ExecutionConfig
 import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.api.common.typeutils.TypeSerializer
 import org.apache.flink.api.java.{DataSet => JavaDataSet}
-import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo, TypeUtils}
+import org.apache.flink.api.scala.typeutils.{CaseClassSerializer, CaseClassTypeInfo, TypeUtils, ScalaNothingTypeInfo}
 
 import _root_.scala.reflect.ClassTag
 import language.experimental.macros
@@ -44,6 +44,11 @@ package object scala {
   // using the Scala API
   implicit def createTypeInformation[T]: TypeInformation[T] = macro TypeUtils.createTypeInfo[T]
 
+  // createTypeInformation does not fire for Nothing in some situations, which is probably
+  // a compiler bug. The following line is a workaround for this.
+  // (See TypeInformationGenTest.testNothingTypeInfoIsAvailableImplicitly)
+  implicit val scalaNothingTypeInfo: TypeInformation[Nothing] = new ScalaNothingTypeInfo()
+
   // We need to wrap Java DataSet because we need the scala operations
   private[flink] def wrap[R: ClassTag](set: JavaDataSet[R]) = new DataSet[R](set)
 

http://git-wip-us.apache.org/repos/asf/flink/blob/6e42f9fc/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfo.scala
----------------------------------------------------------------------
diff --git a/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfo.scala b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfo.scala
new file mode 100644
index 0000000..b59ecf7
--- /dev/null
+++ b/flink-scala/src/main/scala/org/apache/flink/api/scala/typeutils/ScalaNothingTypeInfo.scala
@@ -0,0 +1,47 @@
+/*
+ * 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.ExecutionConfig
+import org.apache.flink.api.common.typeinfo.TypeInformation
+import org.apache.flink.api.common.typeutils.TypeSerializer
+
+class ScalaNothingTypeInfo extends TypeInformation[Nothing] {
+
+  override def isBasicType: Boolean = false
+  override def isTupleType: Boolean = false
+  override def getArity: Int = 0
+  override def getTotalFields: Int = 0
+  override def getTypeClass: Class[Nothing] = classOf[Nothing]
+  override def isKeyType: Boolean = false
+
+  override def createSerializer(config: ExecutionConfig): TypeSerializer[Nothing] =
+    (new NothingSerializer).asInstanceOf[TypeSerializer[Nothing]]
+
+  override def hashCode(): Int = classOf[ScalaNothingTypeInfo].hashCode
+
+  override def toString: String = "ScalaNothingTypeInfo"
+
+  override def equals(obj: Any): Boolean = {
+    obj.isInstanceOf[ScalaNothingTypeInfo]
+  }
+
+  override def canEqual(obj: Any): Boolean = {
+    obj.isInstanceOf[ScalaNothingTypeInfo]
+  }
+}

http://git-wip-us.apache.org/repos/asf/flink/blob/6e42f9fc/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
----------------------------------------------------------------------
diff --git a/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala b/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
index 43c35f9..1170129 100644
--- a/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
+++ b/flink-tests/src/test/scala/org/apache/flink/api/scala/types/TypeInformationGenTest.scala
@@ -576,5 +576,21 @@ class TypeInformationGenTest {
     Assert.assertEquals(caseClassTypeInfo, caseClassInTupleTypeInfo.getTypeAt("_2"))
 
   }
+
+  /**
+   * Tests the "implicit val scalaNothingTypeInfo" in
+   * flink-scala/src/main/scala/org/apache/flink/api/scala/package.scala
+   * This does not compile without that line.
+   */
+  @Test
+  def testNothingTypeInfoIsAvailableImplicitly() : Unit = {
+    def g() = {
+
+      def f[O: TypeInformation](x: O): Unit = {}
+
+      f(???) // O will be Nothing
+    }
+    // (Do not call g, because it throws NotImplementedError. This is a compile time test.)
+  }
 }