You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@flink.apache.org by GitBox <gi...@apache.org> on 2018/08/19 06:43:36 UTC

[GitHub] asfgit closed pull request #6575: [hotfix][table] Fix bug in RowtimeValidator when getting custom TimestampExtractor

asfgit closed pull request #6575: [hotfix][table] Fix bug in RowtimeValidator when getting custom TimestampExtractor
URL: https://github.com/apache/flink/pull/6575
 
 
   

This is a PR merged from a forked repository.
As GitHub hides the original diff on merge, it is displayed below for
the sake of provenance:

As this is a foreign pull request (from a fork), the diff is supplied
below (as it won't show otherwise due to GitHub magic):

diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala
index 160347ec420..4bd51c0158e 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/descriptors/RowtimeValidator.scala
@@ -177,7 +177,7 @@ object RowtimeValidator {
 
       case ROWTIME_TIMESTAMPS_TYPE_VALUE_CUSTOM =>
         val clazz = properties.getClass(
-          ROWTIME_TIMESTAMPS_CLASS,
+          prefix + ROWTIME_TIMESTAMPS_CLASS,
           classOf[TimestampExtractor])
         DescriptorProperties.deserialize(
           properties.getString(prefix + ROWTIME_TIMESTAMPS_SERIALIZED),
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala
index d5930fae64f..9026f174aff 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/descriptors/RowtimeTest.scala
@@ -20,11 +20,15 @@ package org.apache.flink.table.descriptors
 
 import java.util
 
+import org.apache.flink.api.common.typeinfo.TypeInformation
 import org.apache.flink.streaming.api.watermark.Watermark
-import org.apache.flink.table.api.ValidationException
-import org.apache.flink.table.descriptors.RowtimeTest.CustomAssigner
-import org.apache.flink.table.sources.wmstrategies.PunctuatedWatermarkAssigner
+import org.apache.flink.table.api.{Types, ValidationException}
+import org.apache.flink.table.descriptors.RowtimeTest.{CustomAssigner, CustomExtractor}
+import org.apache.flink.table.expressions.{Cast, Expression, ResolvedFieldReference}
+import org.apache.flink.table.sources.tsextractors.TimestampExtractor
+import org.apache.flink.table.sources.wmstrategies.{BoundedOutOfOrderTimestamps, PunctuatedWatermarkAssigner}
 import org.apache.flink.types.Row
+import org.junit.Assert.{assertEquals, assertTrue}
 import org.junit.Test
 
 import scala.collection.JavaConverters._
@@ -46,6 +50,25 @@ class RowtimeTest extends DescriptorTestBase {
     addPropertyAndVerify(descriptors().get(0), "rowtime.watermarks.type", "from-source")
   }
 
+  @Test
+  def testCustomTimestampExtractor(): Unit = {
+    val desc1 = Schema()
+      .field("f1", Types.STRING)
+      .field("f2", Types.STRING)
+      .field("f3", Types.SQL_TIMESTAMP)
+      .field("rt", Types.SQL_TIMESTAMP).rowtime(
+        Rowtime().timestampsFromExtractor(new CustomExtractor("f3"))
+          .watermarksPeriodicBounded(1000L))
+    val props = new DescriptorProperties()
+    desc1.addProperties(props)
+
+    val rowtime = SchemaValidator.deriveRowtimeAttributes(props).get(0)
+    assertEquals("rt", rowtime.getAttributeName)
+    val extractor = rowtime.getTimestampExtractor
+    assertTrue(extractor.equals(new CustomExtractor("f3")))
+    assertTrue(rowtime.getWatermarkStrategy.isInstanceOf[BoundedOutOfOrderTimestamps])
+  }
+
   // ----------------------------------------------------------------------------------------------
 
   override def descriptors(): util.List[Descriptor] = {
@@ -57,7 +80,11 @@ class RowtimeTest extends DescriptorTestBase {
       .timestampsFromSource()
       .watermarksFromStrategy(new CustomAssigner())
 
-    util.Arrays.asList(desc1, desc2)
+    val desc3 = Rowtime()
+      .timestampsFromExtractor(new CustomExtractor("tsField"))
+      .watermarksPeriodicBounded(1000L)
+
+    util.Arrays.asList(desc1, desc2, desc3)
   }
 
   override def validator(): DescriptorValidator = {
@@ -83,7 +110,19 @@ class RowtimeTest extends DescriptorTestBase {
         "F0ZWd5mB_uSxDZ8-MCAAB4cA")
     )
 
-    util.Arrays.asList(props1.asJava, props2.asJava)
+    val props3 = Map(
+      "rowtime.timestamps.type" -> "custom",
+      "rowtime.timestamps.class" -> ("org.apache.flink.table.descriptors." +
+        "RowtimeTest$CustomExtractor"),
+      "rowtime.timestamps.serialized" -> ("rO0ABXNyAD5vcmcuYXBhY2hlLmZsaW5rLnRhYmxlLmRlc2NyaXB0b3" +
+        "JzLlJvd3RpbWVUZXN0JEN1c3RvbUV4dHJhY3RvcoaChjMg55xwAgABTAAFZmllbGR0ABJMamF2YS9sYW5nL1N0cm" +
+        "luZzt4cgA-b3JnLmFwYWNoZS5mbGluay50YWJsZS5zb3VyY2VzLnRzZXh0cmFjdG9ycy5UaW1lc3RhbXBFeHRyYW" +
+        "N0b3LU8E2thK4wMQIAAHhwdAAHdHNGaWVsZA"),
+      "rowtime.watermarks.type" -> "periodic-bounded",
+      "rowtime.watermarks.delay" -> "1000"
+    )
+
+    util.Arrays.asList(props1.asJava, props2.asJava, props3.asJava)
   }
 }
 
@@ -93,4 +132,36 @@ object RowtimeTest {
     override def getWatermark(row: Row, timestamp: Long): Watermark =
       throw new UnsupportedOperationException()
   }
+
+  class CustomExtractor(val field: String) extends TimestampExtractor {
+    def this() = {
+      this("ts")
+    }
+
+    override def getArgumentFields: Array[String] = Array(field)
+
+    override def validateArgumentFields(argumentFieldTypes: Array[TypeInformation[_]]): Unit = {
+      argumentFieldTypes(0) match {
+        case Types.SQL_TIMESTAMP =>
+        case _ =>
+          throw ValidationException(
+            s"Field 'ts' must be of type Timestamp but is of type ${argumentFieldTypes(0)}.")
+      }
+    }
+
+    override def getExpression(fieldAccesses: Array[ResolvedFieldReference]): Expression = {
+      val fieldAccess: Expression = fieldAccesses(0)
+      require(fieldAccess.resultType == Types.SQL_TIMESTAMP)
+      Cast(fieldAccess, Types.LONG)
+    }
+
+    override def equals(other: Any): Boolean = other match {
+      case that: CustomExtractor => field == that.field
+      case _ => false
+    }
+
+    override def hashCode(): Int = {
+      field.hashCode
+    }
+  }
 }


 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services