[
https://issues.apache.org/jira/browse/FLINK-7205?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16579706#comment-16579706
]
ASF GitHub Bot commented on FLINK-7205:
---------------------------------------
asfgit closed pull request #6381: [FLINK-7205] [table] Add UUID supported in
SQL and TableApi
URL: https://github.com/apache/flink/pull/6381
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/docs/dev/table/sql.md b/docs/dev/table/sql.md
index 148f30727d9..4d2cbbadc0a 100644
--- a/docs/dev/table/sql.md
+++ b/docs/dev/table/sql.md
@@ -1664,6 +1664,17 @@ RAND_INTEGER(seed integer, bound integer)
</td>
</tr>
+ <tr>
+ <td>
+ {% highlight text %}
+UUID()
+{% endhighlight %}
+ </td>
+ <td>
+ <p>Returns a uuid string like '3d3c68f7-f608-473f-b60c-b0c44ad4cc4e'
according to RFC 4122.</p>
+ </td>
+ </tr>
+
<tr>
<td>
{% highlight text %}
diff --git a/docs/dev/table/tableApi.md b/docs/dev/table/tableApi.md
index b702dddcde5..a78543bd6c3 100644
--- a/docs/dev/table/tableApi.md
+++ b/docs/dev/table/tableApi.md
@@ -2322,6 +2322,17 @@ randInteger(seed integer, bound integer)
</td>
</tr>
+ <tr>
+ <td>
+ {% highlight java %}
+UUID()
+{% endhighlight %}
+ </td>
+ <td>
+ <p>Returns a uuid string like '3d3c68f7-f608-473f-b60c-b0c44ad4cc4e'
according to RFC 4122.</p>
+ </td>
+ </tr>
+
<tr>
<td>
{% highlight java %}
@@ -3910,6 +3921,17 @@ randInteger(seed integer, bound integer)
</td>
</tr>
+ <tr>
+ <td>
+ {% highlight scala %}
+UUID()
+{% endhighlight %}
+ </td>
+ <td>
+ <p>Returns a uuid string like '3d3c68f7-f608-473f-b60c-b0c44ad4cc4e'
according to RFC 4122.</p>
+ </td>
+ </tr>
+
<tr>
<td>
{% highlight scala %}
diff --git
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
index c7c805f6743..23ac30d3d8b 100644
---
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
+++
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/api/scala/expressionDsl.scala
@@ -1247,4 +1247,17 @@ object concat_ws {
}
}
+/**
+ * Returns the uuid according to RFC 4122.
+ */
+object uuid {
+
+ /**
+ * Returns the uuid according to RFC 4122.
+ */
+ def apply(): Expression = {
+ UUID()
+ }
+}
+
// scalastyle:on object.name
diff --git
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
index 74b69d6afcc..16beea9149d 100644
---
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
+++
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
@@ -158,6 +158,12 @@ object FunctionGenerator {
STRING_TYPE_INFO,
BuiltInMethods.TOBASE64)
+ addSqlFunction(
+ UUID,
+ Seq(),
+ new UUIDCallGen()
+ )
+
//
----------------------------------------------------------------------------------------------
// Arithmetic functions
//
----------------------------------------------------------------------------------------------
diff --git
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/UUIDCallGen.scala
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/UUIDCallGen.scala
new file mode 100644
index 00000000000..537a19857c9
--- /dev/null
+++
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/UUIDCallGen.scala
@@ -0,0 +1,39 @@
+/*
+ * 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.table.codegen.calls
+
+import org.apache.flink.api.common.typeinfo.BasicTypeInfo.STRING_TYPE_INFO
+import
org.apache.flink.table.codegen.calls.CallGenerator.generateCallIfArgsNotNull
+import org.apache.flink.table.codegen.{CodeGenerator, GeneratedExpression}
+
+/**
+ * Generates a UUID function call.
+ */
+class UUIDCallGen extends CallGenerator {
+
+ override def generate(codeGenerator: CodeGenerator,
+ operands: Seq[GeneratedExpression]):
GeneratedExpression = {
+ generateCallIfArgsNotNull(codeGenerator.nullCheck, STRING_TYPE_INFO,
operands) {
+ terms =>
+ """
+ |java.util.UUID.randomUUID().toString()
+ """.stripMargin
+ }
+ }
+}
diff --git
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
index cf3efa938a6..5d8647c3ca6 100644
---
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
+++
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/expressions/mathExpressions.scala
@@ -422,3 +422,11 @@ case class Bin(child: Expression) extends UnaryExpression {
relBuilder.call(ScalarSqlFunctions.BIN, child.toRexNode)
}
}
+
+case class UUID() extends LeafExpression {
+ override private[flink] def resultType = BasicTypeInfo.STRING_TYPE_INFO
+
+ override private[flink] def toRexNode(implicit relBuilder: RelBuilder):
RexNode = {
+ relBuilder.call(ScalarSqlFunctions.UUID)
+ }
+}
diff --git
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ScalarSqlFunctions.scala
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ScalarSqlFunctions.scala
index 21793e3351c..37f1a36e925 100644
---
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ScalarSqlFunctions.scala
+++
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ScalarSqlFunctions.scala
@@ -158,6 +158,17 @@ object ScalarSqlFunctions {
SqlFunctionCategory.STRING
)
+ val UUID = new SqlFunction(
+ "UUID",
+ SqlKind.OTHER_FUNCTION,
+ ReturnTypes.VARCHAR_2000,
+ null,
+ OperandTypes.NILADIC,
+ SqlFunctionCategory.STRING
+ ) {
+ override def isDeterministic: Boolean = false
+ }
+
val DATE_FORMAT = new SqlFunction(
"DATE_FORMAT",
SqlKind.OTHER_FUNCTION,
diff --git
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
index bca156c261a..7127b94e6f9 100644
---
a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
+++
b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
@@ -204,6 +204,7 @@ object FunctionCatalog {
"rpad" -> classOf[Rpad],
"fromBase64" -> classOf[FromBase64],
"toBase64" -> classOf[ToBase64],
+ "uuid" -> classOf[UUID],
// math functions
"plus" -> classOf[Plus],
@@ -451,6 +452,8 @@ class BasicOperatorTable extends ReflectiveSqlOperatorTable
{
ScalarSqlFunctions.SHA2,
ScalarSqlFunctions.FROM_BASE64,
ScalarSqlFunctions.TO_BASE64,
+ ScalarSqlFunctions.UUID,
+
// EXTENSIONS
BasicOperatorTable.TUMBLE,
BasicOperatorTable.HOP,
diff --git
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
index d3b606b1d01..ebe83edb0ae 100644
---
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
+++
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/NonDeterministicTests.scala
@@ -80,6 +80,15 @@ class NonDeterministicTests extends ExpressionTestBase {
"PLEASE CHECK MANUALLY")
}
+ @Ignore
+ @Test
+ def testUUID(): Unit = {
+ testAllApis(
+ uuid(),
+ "uuid()",
+ "UUID",
+ "PLEASE CHECK MANUALLY")
+ }
//
----------------------------------------------------------------------------------------------
override def testData: Row = new Row(0)
diff --git
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
index 6f9a9ae81b9..dcf22b904e4 100644
---
a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
+++
b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
@@ -471,7 +471,7 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
"FROM_BASE64(f33)",
"null")
}
-
+
@Test
def testToBase64(): Unit = {
testAllApis(
@@ -500,6 +500,39 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
"null")
}
+ @Test
+ def testUUID(): Unit = {
+ testAllApis(
+ UUID().charLength(),
+ "UUID().charLength",
+ "CHARACTER_LENGTH(UUID())",
+ "36")
+
+ testAllApis(
+ UUID().substring(9, 1),
+ "UUID().substring(9, 1)",
+ "SUBSTRING(UUID(), 9, 1)",
+ "-")
+
+ testAllApis(
+ UUID().substring(14, 1),
+ "UUID().substring(14, 1)",
+ "SUBSTRING(UUID(), 14, 1)",
+ "-")
+
+ testAllApis(
+ UUID().substring(19, 1),
+ "UUID().substring(19, 1)",
+ "SUBSTRING(UUID(), 19, 1)",
+ "-")
+
+ testAllApis(
+ UUID().substring(24, 1),
+ "UUID().substring(24, 1)",
+ "SUBSTRING(UUID(), 24, 1)",
+ "-")
+ }
+
//
----------------------------------------------------------------------------------------------
// Math functions
//
----------------------------------------------------------------------------------------------
----------------------------------------------------------------
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:
[email protected]
> Add UUID supported in TableAPI/SQL
> ----------------------------------
>
> Key: FLINK-7205
> URL: https://issues.apache.org/jira/browse/FLINK-7205
> Project: Flink
> Issue Type: Sub-task
> Components: Table API & SQL
> Affects Versions: 1.4.0
> Reporter: sunjincheng
> Assignee: buptljy
> Priority: Major
> Labels: pull-request-available
> Fix For: 1.7.0
>
>
> UUID() returns a value that conforms to UUID version 1 as described in RFC
> 4122. The value is a 128-bit number represented as a utf8 string of five
> hexadecimal numbers in aaaaaaaa-bbbb-cccc-dddd-eeeeeeeeeeee format:
> The first three numbers are generated from the low, middle, and high parts of
> a timestamp. The high part also includes the UUID version number.
> The fourth number preserves temporal uniqueness in case the timestamp value
> loses monotonicity (for example, due to daylight saving time).
> The fifth number is an IEEE 802 node number that provides spatial uniqueness.
> A random number is substituted if the latter is not available (for example,
> because the host device has no Ethernet card, or it is unknown how to find
> the hardware address of an interface on the host operating system). In this
> case, spatial uniqueness cannot be guaranteed. Nevertheless, a collision
> should have very low probability.
> See: [RFC 4122:
> http://www.ietf.org/rfc/rfc4122.txt|http://www.ietf.org/rfc/rfc4122.txt]
> See detailed semantics:
> MySql:
> [https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_uuid|https://dev.mysql.com/doc/refman/5.7/en/miscellaneous-functions.html#function_uuid]
> Welcome anybody feedback -:).
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)