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

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

                Author: ASF GitHub Bot
            Created on: 31/May/19 18:28
            Start Date: 31/May/19 18:28
    Worklog Time Spent: 10m 
      Work Description: reuvenlax commented on pull request #8725: [BEAM-6674] 
Add schema support to JdbcIO read
URL: https://github.com/apache/beam/pull/8725#discussion_r289435182
 
 

 ##########
 File path: 
sdks/java/io/jdbc/src/main/java/org/apache/beam/sdk/io/jdbc/SchemaUtil.java
 ##########
 @@ -0,0 +1,321 @@
+/*
+ * 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.beam.sdk.io.jdbc;
+
+import java.io.Serializable;
+import java.sql.Array;
+import java.sql.Date;
+import java.sql.JDBCType;
+import java.sql.ResultSet;
+import java.sql.ResultSetMetaData;
+import java.sql.SQLException;
+import java.sql.Time;
+import java.sql.Timestamp;
+import java.time.Instant;
+import java.time.LocalDate;
+import java.time.LocalTime;
+import java.time.ZoneOffset;
+import java.time.ZonedDateTime;
+import java.util.ArrayList;
+import java.util.EnumMap;
+import java.util.List;
+import java.util.stream.Collectors;
+import java.util.stream.IntStream;
+import org.apache.beam.sdk.schemas.LogicalTypes;
+import org.apache.beam.sdk.schemas.Schema;
+import org.apache.beam.sdk.values.Row;
+import 
org.apache.beam.vendor.guava.v20_0.com.google.common.annotations.VisibleForTesting;
+import 
org.apache.beam.vendor.guava.v20_0.com.google.common.collect.ImmutableMap;
+import org.joda.time.DateTime;
+import org.joda.time.chrono.ISOChronology;
+
+/** Provides utility functions for working with Beam {@link Schema} types. */
+class SchemaUtil {
+  private static final String SQL_DATE = "SqlDateType";
+  private static final String SQL_TIME = "SqlTimeType";
+  private static final String SQL_TIMESTAMP_WITH_LOCAL_TZ = 
"SqlTimestampWithLocalTzType";
+
+  @VisibleForTesting
+  static final Schema.FieldType SQL_DATE_LOGICAL_TYPE =
+      Schema.FieldType.logicalType(
+          new LogicalTypes.PassThroughLogicalType<Instant>(
+              SQL_DATE, "", Schema.FieldType.DATETIME) {});
+
+  @VisibleForTesting
+  static final Schema.FieldType SQL_TIME_LOGICAL_TYPE =
+      Schema.FieldType.logicalType(
+          new LogicalTypes.PassThroughLogicalType<Instant>(
+              SQL_TIME, "", Schema.FieldType.DATETIME) {});
+
+  @VisibleForTesting
+  static final Schema.FieldType SQL_TIMESTAMP_WITH_LOCAL_TZ_LOGICAL_TYPE =
+      Schema.FieldType.logicalType(
+          new LogicalTypes.PassThroughLogicalType<Instant>(
+              SQL_TIMESTAMP_WITH_LOCAL_TZ, "", Schema.FieldType.DATETIME) {});
+
+  /**
+   * Interface implemented by functions that extract values of different types 
from a JDBC
+   * ResultSet.
+   */
+  @FunctionalInterface
+  interface ResultSetFieldExtractor extends Serializable {
+    Object extract(ResultSet rs, Integer index) throws SQLException;
+  }
+
+  // ResultSetExtractors for primitive schema types (excluding arrays, structs 
and logical types).
+  private static final EnumMap<Schema.TypeName, ResultSetFieldExtractor>
+      RESULTSET_FIELD_EXTRACTORS =
+          new EnumMap<>(
+              ImmutableMap.<Schema.TypeName, ResultSetFieldExtractor>builder()
+                  .put(Schema.TypeName.BOOLEAN, ResultSet::getBoolean)
+                  .put(Schema.TypeName.BYTE, ResultSet::getByte)
+                  .put(Schema.TypeName.BYTES, ResultSet::getBytes)
+                  .put(Schema.TypeName.DATETIME, ResultSet::getTimestamp)
+                  .put(Schema.TypeName.DECIMAL, ResultSet::getBigDecimal)
+                  .put(Schema.TypeName.DOUBLE, ResultSet::getDouble)
+                  .put(Schema.TypeName.FLOAT, ResultSet::getFloat)
+                  .put(Schema.TypeName.INT16, ResultSet::getShort)
+                  .put(Schema.TypeName.INT32, ResultSet::getInt)
+                  .put(Schema.TypeName.INT64, ResultSet::getLong)
+                  .put(Schema.TypeName.STRING, ResultSet::getString)
+                  .build());
+
+  private static final ResultSetFieldExtractor DATE_EXTRACTOR = 
createDateExtractor();
+  private static final ResultSetFieldExtractor TIME_EXTRACTOR = 
createTimeExtractor();
+  private static final ResultSetFieldExtractor TIMESTAMP_EXTRACTOR = 
createTimestampExtractor();
+
+  /*
+   * Mapping of primitive JDBC types to Beam schema types.
+   * TODO: Add support for BLOB and CLOB types
+   */
+  private static final EnumMap<JDBCType, Schema.FieldType> 
JDBC_TO_BEAM_MAPPING =
+      new EnumMap<>(
+          ImmutableMap.<JDBCType, Schema.FieldType>builder()
+              .put(JDBCType.BIGINT, Schema.FieldType.INT64)
+              .put(JDBCType.BINARY, Schema.FieldType.BYTES)
+              .put(JDBCType.BIT, Schema.FieldType.BOOLEAN)
+              .put(JDBCType.BOOLEAN, Schema.FieldType.BOOLEAN)
+              .put(JDBCType.CHAR, Schema.FieldType.STRING)
+              .put(JDBCType.DECIMAL, Schema.FieldType.DECIMAL)
+              .put(JDBCType.DOUBLE, Schema.FieldType.DOUBLE)
+              .put(JDBCType.FLOAT, Schema.FieldType.FLOAT)
+              .put(JDBCType.INTEGER, Schema.FieldType.INT32)
+              .put(JDBCType.LONGVARBINARY, Schema.FieldType.BYTES)
+              .put(JDBCType.LONGVARCHAR, Schema.FieldType.STRING)
+              .put(JDBCType.NUMERIC, Schema.FieldType.DECIMAL)
+              .put(JDBCType.REAL, Schema.FieldType.FLOAT)
+              .put(JDBCType.SMALLINT, Schema.FieldType.INT16)
+              .put(JDBCType.TIMESTAMP, Schema.FieldType.DATETIME)
+              .put(JDBCType.TINYINT, Schema.FieldType.BYTE)
+              .put(JDBCType.VARBINARY, Schema.FieldType.BYTES)
+              .put(JDBCType.VARCHAR, Schema.FieldType.STRING)
+              .build());
+
+  /**
+   * Interface implemented by functions that create Beam {@link
+   * org.apache.beam.sdk.schemas.Schema.Field} corresponding to JDBC field 
metadata.
+   */
+  @FunctionalInterface
+  interface BeamFieldConverter extends Serializable {
+    Schema.Field create(int index, ResultSetMetaData md) throws SQLException;
+  }
+
+  /**
+   * Infers the Beam {@link Schema} from {@link ResultSetMetaData}.
+   *
+   * <p>Only a subset of JDBC types are supported.
+   */
+  static Schema toBeamSchema(ResultSetMetaData md) throws SQLException {
+    Schema.Builder schemaBuilder = Schema.builder();
+    for (int i = 1; i <= md.getColumnCount(); i++) {
+      JDBCType jdbcType = JDBCType.valueOf(md.getColumnType(i));
+      Schema.Field field;
+      switch (jdbcType) {
+        case ARRAY:
+          field = beamArrayField().create(i, md);
+          break;
+        case DATE:
+          field = beamFieldOfType(SQL_DATE_LOGICAL_TYPE).create(i, md);
+          break;
+        case TIME:
+          field = beamFieldOfType(SQL_TIME_LOGICAL_TYPE).create(i, md);
+          break;
+        case TIMESTAMP_WITH_TIMEZONE:
+          field = 
beamFieldOfType(SQL_TIMESTAMP_WITH_LOCAL_TZ_LOGICAL_TYPE).create(i, md);
+          break;
+        default:
+          if (!JDBC_TO_BEAM_MAPPING.containsKey(jdbcType)) {
 
 Review comment:
   Are the unsupported types fundamentally unsupportable? If not, we should 
file a JIRA to support them (probably via logical types)
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

    Worklog Id:     (was: 251954)
    Time Spent: 1h  (was: 50m)

> The JdbcIO source should produce schemas
> ----------------------------------------
>
>                 Key: BEAM-6674
>                 URL: https://issues.apache.org/jira/browse/BEAM-6674
>             Project: Beam
>          Issue Type: Sub-task
>          Components: io-java-jdbc
>            Reporter: Reuven Lax
>            Assignee: Shehzaad Nakhoda
>            Priority: Major
>          Time Spent: 1h
>  Remaining Estimate: 0h
>




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

Reply via email to