[ 
https://issues.apache.org/jira/browse/BEAM-4700?focusedWorklogId=124127&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-124127
 ]

ASF GitHub Bot logged work on BEAM-4700:
----------------------------------------

                Author: ASF GitHub Bot
            Created on: 17/Jul/18 16:23
            Start Date: 17/Jul/18 16:23
    Worklog Time Spent: 10m 
      Work Description: kennknowles closed pull request #5919: [BEAM-4700] 
[SQL] Default timezone is UTC
URL: https://github.com/apache/beam/pull/5919
 
 
   

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/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
index 75a2e901eba..dd951d5ab00 100644
--- 
a/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
+++ 
b/sdks/java/extensions/sql/src/main/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriver.java
@@ -30,6 +30,7 @@
 import org.apache.beam.sdk.extensions.sql.meta.provider.TableProvider;
 import org.apache.beam.sdk.options.PipelineOptions;
 import org.apache.beam.sdk.util.ReleaseInfo;
+import org.apache.calcite.avatica.BuiltInConnectionProperty;
 import org.apache.calcite.avatica.ConnectStringParser;
 import org.apache.calcite.avatica.ConnectionProperty;
 import org.apache.calcite.config.CalciteConnectionProperty;
@@ -86,6 +87,7 @@ public Connection connect(String url, Properties info) throws 
SQLException {
     final BeamCalciteSchema beamCalciteSchema = (BeamCalciteSchema) 
info.get(BEAM_CALCITE_SCHEMA);
 
     Properties info2 = new Properties(info);
+    setDefault(info2, BuiltInConnectionProperty.TIME_ZONE, "UTC");
     setDefault(info2, CalciteConnectionProperty.LEX, Lex.JAVA.name());
     setDefault(
         info2,
diff --git 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
index f5215c9a8d3..bb96b300ee7 100644
--- 
a/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
+++ 
b/sdks/java/extensions/sql/src/test/java/org/apache/beam/sdk/extensions/sql/impl/JdbcDriverTest.java
@@ -206,7 +206,6 @@ public void testSelectsFromExistingTable() throws Exception 
{
   }
 
   @Test
-  @Ignore("https://issues.apache.org/jira/browse/CALCITE-2394";)
   public void testTimestampWithDefaultTimezone() throws Exception {
     TestTableProvider tableProvider = new TestTableProvider();
     Connection connection = JdbcDriver.connect(tableProvider);


 

----------------------------------------------------------------
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]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 124127)
    Time Spent: 2h 50m  (was: 2h 40m)

> JDBC driver cannot support TIMESTAMP data type
> ----------------------------------------------
>
>                 Key: BEAM-4700
>                 URL: https://issues.apache.org/jira/browse/BEAM-4700
>             Project: Beam
>          Issue Type: Bug
>          Components: dsl-sql
>            Reporter: Kenneth Knowles
>            Assignee: Andrew Pilloud
>            Priority: Major
>          Time Spent: 2h 50m
>  Remaining Estimate: 0h
>
> Avatica allows column representation to be customized, so a timestamp can be 
> stored as a variety of types. Joda ReadableInstant is none of these types: 
> https://github.com/apache/calcite-avatica/blob/acb675de97b9b0743c09368820a770e2ceda05f8/core/src/main/java/org/apache/calcite/avatica/util/AbstractCursor.java#L162
> By default, it seems to be configured to store {{TIMESTAMP}} columns as 
> {{long}} values. If you run the SQL shell and select a {{TIMESTAMP}} column, 
> you get:
> {code}
> ava.lang.ClassCastException: org.joda.time.Instant cannot be cast to 
> java.lang.Number
>         at 
> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.avatica.util.AbstractCursor$NumberAccessor.getNumber(AbstractCursor.java:726)
>         at 
> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.avatica.util.AbstractCursor$TimestampFromNumberAccessor.getString(AbstractCursor.java:1026)
>         at 
> org.apache.beam.repackaged.beam_sdks_java_extensions_sql.org.apache.calcite.avatica.AvaticaResultSet.getString(AvaticaResultSet.java:225)
>         at sqlline.Rows$Row.<init>(Rows.java:183)
> {code}
> So, essentially, Beam SQL Shell does not support timestamps.
> We may be able to:
>  - override how the accessor for our existing storage is created
>  - configure what the column representation is (this doesn't really help, 
> since none of the choices are ours)
>  - convert timestamps to longs in BeamEnumerableConverter; not sure how many 
> conversions will be required here



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to