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:42:07 UTC

[kyuubi] branch branch-1.7 updated (74477bb62 -> e08245926)

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

chengpan pushed a change to branch branch-1.7
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


 discard 74477bb62 [KYUUBI #4635] [Kyuubi #1704] Support flink time type in query operation
     new e08245926 [KYUUBI #4635] Support flink time type in query operation

This update added new revisions after undoing existing revisions.
That is to say, some revisions that were in the old version of the
branch are not in the new version.  This situation occurs
when a user --force pushes a change and generates a repository
containing something like this:

 * -- * -- B -- O -- O -- O   (74477bb62)
            \
             N -- N -- N   refs/heads/branch-1.7 (e08245926)

You should already have received notification emails for all of the O
revisions, and so the following emails describe only the N revisions
from the common base, B.

Any revisions marked "omit" are not gone; other references still
refer to them.  Any revisions marked "discard" are gone forever.

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:


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

Posted by ch...@apache.org.
This is an automated email from the ASF dual-hosted git repository.

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

commit e082459268bd5f4bc720c19617aefef2ae21072c
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  #1770, 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>
    (cherry picked from commit 1f8f0e68e7258599f115703f89baf598629a4e9a)
    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 8ec794ffb..976b39c2b 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._
@@ -205,6 +205,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 70eb84ddb..0b628010c 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
@@ -756,6 +756,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 =
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)
   }