You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kyuubi.apache.org by ch...@apache.org on 2023/03/30 11:41:25 UTC

[kyuubi] 01/01: [KYUUBI #4635] Support flink time type in query operation

This is an automated email from the ASF dual-hosted git repository.

chengpan pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kyuubi.git

commit e7a5d2014ecbca67f4b4a46cb51b29ef8a3b28a6
Author: Ruguo Yu <ji...@163.com>
AuthorDate: Thu Mar 30 19:38:44 2023 +0800

    [KYUUBI #4635] Support flink time type in query operation
    
    Followup #1704, support flink `time` type in query operation
    
    - [x] Add some test cases that check the changes thoroughly including negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [x] [Run test](https://kyuubi.readthedocs.io/en/master/develop_tools/testing.html#running-tests) locally before make a pull request
    
    Closes #4635 from yuruguo/support-flink-time-type.
    
    Closes #4635
    
    9f9a3e72d [Ruguo Yu] [Kyuubi #1704] Support flink time type in query operation
    
    Authored-by: Ruguo Yu <ji...@163.com>
    Signed-off-by: Cheng Pan <ch...@apache.org>
---
 .../kyuubi/engine/flink/operation/ExecuteStatement.scala   |  5 ++++-
 .../org/apache/kyuubi/engine/flink/schema/RowSet.scala     |  1 +
 .../engine/flink/operation/FlinkOperationSuite.scala       | 14 ++++++++++++++
 .../main/scala/org/apache/kyuubi/util/RowSetUtils.scala    | 12 +++++++++++-
 4 files changed, 30 insertions(+), 2 deletions(-)

diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala
index db7876b6c..0438b98d1 100644
--- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala
+++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/operation/ExecuteStatement.scala
@@ -17,7 +17,7 @@
 
 package org.apache.kyuubi.engine.flink.operation
 
-import java.time.LocalDate
+import java.time.{LocalDate, LocalTime}
 import java.util
 
 import scala.collection.JavaConverters._
@@ -190,6 +190,9 @@ class ExecuteStatement(
         case _: DateType =>
           val date = RowSetUtils.formatLocalDate(LocalDate.ofEpochDay(r.getInt(i)))
           row.setField(i, date)
+        case _: TimeType =>
+          val time = RowSetUtils.formatLocalTime(LocalTime.ofNanoOfDay(r.getLong(i) * 1000 * 1000))
+          row.setField(i, time)
         case t: TimestampType =>
           val ts = RowSetUtils
             .formatLocalDateTime(r.getTimestamp(i, t.getPrecision)
diff --git a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala
index 2b3ae50b7..ad83f9c2b 100644
--- a/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala
+++ b/externals/kyuubi-flink-sql-engine/src/main/scala/org/apache/kyuubi/engine/flink/schema/RowSet.scala
@@ -307,6 +307,7 @@ object RowSet {
     case _: MapType => TTypeId.MAP_TYPE
     case _: RowType => TTypeId.STRUCT_TYPE
     case _: BinaryType => TTypeId.BINARY_TYPE
+    case _: TimeType => TTypeId.STRING_TYPE
     case t @ (_: ZonedTimestampType | _: LocalZonedTimestampType | _: MultisetType |
         _: YearMonthIntervalType | _: DayTimeIntervalType) =>
       throw new IllegalArgumentException(
diff --git a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala
index a38476f4d..5026fd411 100644
--- a/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala
+++ b/externals/kyuubi-flink-sql-engine/src/test/scala/org/apache/kyuubi/engine/flink/operation/FlinkOperationSuite.scala
@@ -755,6 +755,20 @@ class FlinkOperationSuite extends WithFlinkSQLEngine with HiveJDBCTestHelper {
     }
   }
 
+  test("execute statement - select time") {
+    withJdbcStatement() { statement =>
+      val resultSet =
+        statement.executeQuery(
+          "select time '00:00:03', time '00:00:05.123456789'")
+      val metaData = resultSet.getMetaData
+      assert(metaData.getColumnType(1) === java.sql.Types.VARCHAR)
+      assert(metaData.getColumnType(2) === java.sql.Types.VARCHAR)
+      assert(resultSet.next())
+      assert(resultSet.getString(1) == "00:00:03")
+      assert(resultSet.getString(2) == "00:00:05.123")
+    }
+  }
+
   test("execute statement - select array") {
     withJdbcStatement() { statement =>
       val resultSet = statement.executeQuery("select array ['v1', 'v2', 'v3']")
diff --git a/kyuubi-common/src/main/scala/org/apache/kyuubi/util/RowSetUtils.scala b/kyuubi-common/src/main/scala/org/apache/kyuubi/util/RowSetUtils.scala
index fca79c0f2..f320fd902 100644
--- a/kyuubi-common/src/main/scala/org/apache/kyuubi/util/RowSetUtils.scala
+++ b/kyuubi-common/src/main/scala/org/apache/kyuubi/util/RowSetUtils.scala
@@ -18,7 +18,7 @@
 package org.apache.kyuubi.util
 
 import java.nio.ByteBuffer
-import java.time.{Instant, LocalDate, LocalDateTime, ZoneId}
+import java.time.{Instant, LocalDate, LocalDateTime, LocalTime, ZoneId}
 import java.time.chrono.IsoChronology
 import java.time.format.DateTimeFormatterBuilder
 import java.time.temporal.ChronoField
@@ -41,6 +41,12 @@ private[kyuubi] object RowSetUtils {
 
   private lazy val legacyDateFormatter = FastDateFormat.getInstance("yyyy-MM-dd", Locale.US)
 
+  private lazy val timeFormatter = createDateTimeFormatterBuilder()
+    .appendPattern("HH:mm:ss")
+    .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)
+    .toFormatter(Locale.US)
+    .withChronology(IsoChronology.INSTANCE)
+
   private lazy val timestampFormatter = createDateTimeFormatterBuilder()
     .appendPattern("yyyy-MM-dd HH:mm:ss")
     .appendFraction(ChronoField.NANO_OF_SECOND, 0, 9, true)
@@ -59,6 +65,10 @@ private[kyuubi] object RowSetUtils {
     dateFormatter.format(ld)
   }
 
+  def formatLocalTime(lt: LocalTime): String = {
+    timeFormatter.format(lt)
+  }
+
   def formatLocalDateTime(ldt: LocalDateTime): String = {
     timestampFormatter.format(ldt)
   }