You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@flink.apache.org by tw...@apache.org on 2018/01/10 16:42:14 UTC
[2/2] flink git commit: [FLINK-6893] [table] Add BIN function support
[FLINK-6893] [table] Add BIN function support
This closes #4128.
Project: http://git-wip-us.apache.org/repos/asf/flink/repo
Commit: http://git-wip-us.apache.org/repos/asf/flink/commit/2914e596
Tree: http://git-wip-us.apache.org/repos/asf/flink/tree/2914e596
Diff: http://git-wip-us.apache.org/repos/asf/flink/diff/2914e596
Branch: refs/heads/master
Commit: 2914e596bf2af968197b5241aa40840e2e9408ce
Parents: d42759d
Author: sunjincheng121 <su...@gmail.com>
Authored: Thu Jun 15 14:30:25 2017 +0800
Committer: twalthr <tw...@apache.org>
Committed: Wed Jan 10 17:41:50 2018 +0100
----------------------------------------------------------------------
docs/dev/table/sql.md | 10 +++++++++
.../table/codegen/calls/BuiltInMethods.scala | 1 +
.../table/codegen/calls/FunctionGenerator.scala | 11 ++++++++++
.../functions/sql/ScalarSqlFunctions.scala | 9 ++++++++
.../flink/table/validate/FunctionCatalog.scala | 1 +
.../table/expressions/ScalarFunctionsTest.scala | 12 ++++++++++
.../ScalarFunctionsValidationTest.scala | 23 ++++++++++++++++----
7 files changed, 63 insertions(+), 4 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/docs/dev/table/sql.md
----------------------------------------------------------------------
diff --git a/docs/dev/table/sql.md b/docs/dev/table/sql.md
index 1cb0cdc..4860a7d 100644
--- a/docs/dev/table/sql.md
+++ b/docs/dev/table/sql.md
@@ -1746,6 +1746,16 @@ CONCAT_WS(separator, string1, string2,...)
<p>Returns the string that results from concatenating the arguments using a separator. The separator is added between the strings to be concatenated. Returns NULL If the separator is NULL. CONCAT_WS() does not skip empty strings. However, it does skip any NULL argument. E.g. <code>CONCAT_WS("~", "AA", "BB", "", "CC")</code> returns <code>AA~BB~~CC</code></p>
</td>
</tr>
+ <tr>
+ <td>
+{% highlight text %}
+BIN(numeric)
+ {% endhighlight %}
+ </td>
+ <td>
+ <p>Returns a string representation of the binary value of Numeric, Returns NULL if Numeric is NULL.</p>
+ </td>
+ </tr>
</tbody>
</table>
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
index 671ae07..6791e1b 100644
--- a/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
+++ b/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/BuiltInMethods.scala
@@ -96,4 +96,5 @@ object BuiltInMethods {
Types.lookupMethod(
classOf[ScalarFunctions], "concat_ws", classOf[String], classOf[Array[String]])
+ val BIN = Types.lookupMethod(classOf[JLong], "toBinaryString", classOf[Long])
}
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/codegen/calls/FunctionGenerator.scala
----------------------------------------------------------------------
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 b9f1016..7bec58e 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
@@ -413,6 +413,11 @@ object FunctionGenerator {
DOUBLE_TYPE_INFO,
BuiltInMethods.LOG_WITH_BASE)
+ addSqlFunction(
+ ScalarSqlFunctions.E,
+ Seq(),
+ new ConstantCallGen(DOUBLE_TYPE_INFO, Math.E.toString))
+
// ----------------------------------------------------------------------------------------------
// Temporal functions
// ----------------------------------------------------------------------------------------------
@@ -502,6 +507,12 @@ object FunctionGenerator {
new DateFormatCallGen
)
+ addSqlFunctionMethod(
+ ScalarSqlFunctions.BIN,
+ Seq(LONG_TYPE_INFO),
+ STRING_TYPE_INFO,
+ BuiltInMethods.BIN)
+
// ----------------------------------------------------------------------------------------------
// Cryptographic Hash functions
// ----------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/functions/sql/ScalarSqlFunctions.scala
----------------------------------------------------------------------
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 2228e11..8bdd69a 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
@@ -15,6 +15,7 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
+
package org.apache.flink.table.functions.sql
import org.apache.calcite.sql.{SqlFunction, SqlFunctionCategory, SqlKind}
@@ -33,6 +34,14 @@ object ScalarSqlFunctions {
OperandTypes.NILADIC,
SqlFunctionCategory.NUMERIC)
+ val BIN = new SqlFunction(
+ "BIN",
+ SqlKind.OTHER_FUNCTION,
+ ReturnTypes.explicit(SqlTypeName.VARCHAR),
+ null,
+ OperandTypes.family(SqlTypeFamily.INTEGER),
+ SqlFunctionCategory.NUMERIC)
+
val CONCAT = new SqlFunction(
"CONCAT",
SqlKind.OTHER_FUNCTION,
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/flink-libraries/flink-table/src/main/scala/org/apache/flink/table/validate/FunctionCatalog.scala
----------------------------------------------------------------------
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 6d7e24b..ed565a8 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
@@ -416,6 +416,7 @@ class BasicOperatorTable extends ReflectiveSqlOperatorTable {
SqlStdOperatorTable.RAND_INTEGER,
ScalarSqlFunctions.CONCAT,
ScalarSqlFunctions.CONCAT_WS,
+ ScalarSqlFunctions.BIN,
SqlStdOperatorTable.TIMESTAMP_ADD,
ScalarSqlFunctions.LOG,
ScalarSqlFunctions.MD5,
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/ScalarFunctionsTest.scala
----------------------------------------------------------------------
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 643decc..da46e62 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
@@ -352,6 +352,18 @@ class ScalarFunctionsTest extends ScalarTypesTestBase {
"Flink~~~~xx")
}
+ @Test
+ def testBin(): Unit = {
+ testSqlApi("BIN(f2)", "101010")
+ testSqlApi("BIN(f3)", "101011")
+ testSqlApi("BIN(f4)", "101100")
+ testSqlApi("BIN(f7)", "11")
+ testSqlApi("BIN(12)", "1100")
+ testSqlApi("BIN(10)", "1010")
+ testSqlApi("BIN(0)", "0")
+ testSqlApi("BIN(f32)","1111111111111111111111111111111111111111111111111111111111111111")
+ }
+
// ----------------------------------------------------------------------------------------------
// Math functions
// ----------------------------------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/flink/blob/2914e596/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala
----------------------------------------------------------------------
diff --git a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala
index b98c0ab..4a5b740 100644
--- a/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala
+++ b/flink-libraries/flink-table/src/test/scala/org/apache/flink/table/expressions/validation/ScalarFunctionsValidationTest.scala
@@ -39,7 +39,7 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
}
@Test(expected = classOf[IllegalArgumentException])
- def testInvalidLog2(): Unit ={
+ def testInvalidLog2(): Unit = {
// invalid arithmetic argument
testSqlApi(
"LOG(-1)",
@@ -68,17 +68,17 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
// ----------------------------------------------------------------------------------------------
@Test(expected = classOf[SqlParserException])
- def testTimestampAddWithWrongTimestampInterval(): Unit ={
+ def testTimestampAddWithWrongTimestampInterval(): Unit = {
testSqlApi("TIMESTAMPADD(XXX, 1, timestamp '2016-02-24'))", "2016-06-16")
}
@Test(expected = classOf[SqlParserException])
- def testTimestampAddWithWrongTimestampFormat(): Unit ={
+ def testTimestampAddWithWrongTimestampFormat(): Unit = {
testSqlApi("TIMESTAMPADD(YEAR, 1, timestamp '2016-02-24'))", "2016-06-16")
}
@Test(expected = classOf[ValidationException])
- def testTimestampAddWithWrongQuantity(): Unit ={
+ def testTimestampAddWithWrongQuantity(): Unit = {
testSqlApi("TIMESTAMPADD(YEAR, 1.0, timestamp '2016-02-24 12:42:25')", "2016-06-16")
}
@@ -112,4 +112,19 @@ class ScalarFunctionsValidationTest extends ScalarTypesTestBase {
"true"
)
}
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidBin1(): Unit = {
+ testSqlApi("BIN(f12)", "101010") // float type
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidBin2(): Unit = {
+ testSqlApi("BIN(f15)", "101010") // BigDecimal type
+ }
+
+ @Test(expected = classOf[ValidationException])
+ def testInvalidBin3(): Unit = {
+ testSqlApi("BIN(f16)", "101010") // Date type
+ }
}