Github user rxin commented on a diff in the pull request:

    https://github.com/apache/spark/pull/4261#discussion_r23903163
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala 
---
    @@ -0,0 +1,417 @@
    +/*
    + * 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.spark.sql.jdbc
    +
    +import java.sql.{Connection, DatabaseMetaData, DriverManager, ResultSet, 
ResultSetMetaData, SQLException}
    +import scala.collection.mutable.ArrayBuffer
    +
    +import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
    +import org.apache.spark.rdd.RDD
    +import org.apache.spark.util.NextIterator
    +import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion
    +import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow}
    +import org.apache.spark.sql.types._
    +import org.apache.spark.sql.sources._
    +
    +private[sql] object JDBCRDD extends Logging {
    +  /**
    +   * Maps a JDBC type to a Catalyst type.  This function is called only 
when
    +   * the DriverQuirks class corresponding to your database driver returns 
null.
    +   *
    +   * @param sqlType - A field of java.sql.Types
    +   * @return The Catalyst type corresponding to sqlType.
    +   */
    +  private def getCatalystType(sqlType: Int): DataType = {
    +    val answer = sqlType match {
    +      case java.sql.Types.ARRAY         => null
    +      case java.sql.Types.BIGINT        => LongType
    +      case java.sql.Types.BINARY        => BinaryType
    +      case java.sql.Types.BIT           => BooleanType // Per JDBC; Quirks 
handles quirky drivers.
    +      case java.sql.Types.BLOB          => BinaryType
    +      case java.sql.Types.BOOLEAN       => BooleanType
    +      case java.sql.Types.CHAR          => StringType
    +      case java.sql.Types.CLOB          => StringType
    +      case java.sql.Types.DATALINK      => null
    +      case java.sql.Types.DATE          => DateType
    +      case java.sql.Types.DECIMAL       => DecimalType.Unlimited
    +      case java.sql.Types.DISTINCT      => null
    +      case java.sql.Types.DOUBLE        => DoubleType
    +      case java.sql.Types.FLOAT         => FloatType
    +      case java.sql.Types.INTEGER       => IntegerType
    +      case java.sql.Types.JAVA_OBJECT   => null
    +      case java.sql.Types.LONGNVARCHAR  => StringType
    +      case java.sql.Types.LONGVARBINARY => BinaryType
    +      case java.sql.Types.LONGVARCHAR   => StringType
    +      case java.sql.Types.NCHAR         => StringType
    +      case java.sql.Types.NCLOB         => StringType
    +      case java.sql.Types.NULL          => null
    +      case java.sql.Types.NUMERIC       => DecimalType.Unlimited
    +      case java.sql.Types.OTHER         => null
    +      case java.sql.Types.REAL          => DoubleType
    +      case java.sql.Types.REF           => StringType
    +      case java.sql.Types.ROWID         => LongType
    +      case java.sql.Types.SMALLINT      => IntegerType
    +      case java.sql.Types.SQLXML        => StringType
    +      case java.sql.Types.STRUCT        => StringType
    +      case java.sql.Types.TIME          => TimestampType
    +      case java.sql.Types.TIMESTAMP     => TimestampType
    +      case java.sql.Types.TINYINT       => IntegerType
    +      case java.sql.Types.VARBINARY     => BinaryType
    +      case java.sql.Types.VARCHAR       => StringType
    +      case _ => null
    +    }
    +
    +    if (answer == null) throw new SQLException("Unsupported type " + 
sqlType)
    +    answer
    +  }
    +
    +  /**
    +   * Takes a (schema, table) specification and returns the table's Catalyst
    +   * schema.
    +   *
    +   * @param url - The JDBC url to fetch information from.
    +   * @param table - The table name of the desired table.  This may also be 
a
    +   *   SQL query wrapped in parentheses.
    +   *
    +   * @return A StructType giving the table's Catalyst schema.
    +   * @throws SQLException if the table specification is garbage.
    +   * @throws SQLException if the table contains an unsupported type.
    +   */
    +  def resolveTable(url: String, table: String): StructType = {
    +    val quirks = DriverQuirks.get(url)
    +    val conn: Connection = DriverManager.getConnection(url)
    +    try {
    +      val rs = conn.prepareStatement(s"SELECT * FROM $table WHERE 
1=0").executeQuery()
    +      try {
    +        val rsmd = rs.getMetaData
    +        val ncols = rsmd.getColumnCount
    +        var fields = new Array[StructField](ncols);
    +        var i = 0
    +        while (i < ncols) {
    +          val columnName = rsmd.getColumnName(i + 1)
    +          val dataType = rsmd.getColumnType(i + 1)
    +          val typeName = rsmd.getColumnTypeName(i + 1)
    +          val fieldSize = rsmd.getPrecision(i + 1)
    +          val nullable = rsmd.isNullable(i + 1) != 
ResultSetMetaData.columnNoNulls
    +          val metadata = new MetadataBuilder().putString("name", 
columnName)
    +          var columnType = quirks.getCatalystType(dataType, typeName, 
fieldSize, metadata)
    +          if (columnType == null) columnType = getCatalystType(dataType)
    +          fields(i) = StructField(columnName, columnType, nullable, 
metadata.build())
    +          i = i + 1
    +        }
    +        return new StructType(fields)
    +      } finally {
    +        rs.close()
    +      }
    +    } finally {
    +      conn.close()
    +    }
    +
    +    throw new RuntimeException("This line is unreachable.")
    +  }
    +
    +  /**
    +   * Prune all but the specified columns from the specified Catalyst 
schema.
    +   *
    +   * @param schema - The Catalyst schema of the master table
    +   * @param columns - The list of desired columns
    +   *
    +   * @return A Catalyst schema corresponding to columns in the given order.
    +   */
    +  private def pruneSchema(schema: StructType, columns: Array[String]): 
StructType = {
    +    val fieldMap = Map(schema.fields map { x => 
x.metadata.getString("name") -> x }: _*)
    +    new StructType(columns map { name => fieldMap(name) })
    +  }
    +
    +  /**
    +   * Given a driver string and an url, return a function that loads the
    +   * specified driver string then returns a connection to the JDBC url.
    +   * getConnector is run on the driver code, while the function it returns
    +   * is run on the executor.
    +   *
    +   * @param driver - The class name of the JDBC driver for the given url.
    +   * @param url - The JDBC url to connect to.
    +   *
    +   * @return A function that loads the driver and connects to the url.
    +   */
    +  def getConnector(driver: String, url: String): () => Connection = {
    +    () => {
    +      try {
    +        if (driver != null) Class.forName(driver)
    +      } catch {
    +        case e: ClassNotFoundException => {
    +          logWarning(s"Couldn't find class $driver", e);
    +        }
    +      }
    +      DriverManager.getConnection(url)
    +    }
    +  }
    +  /**
    +   * Build and return JDBCRDD from the given information.
    +   *
    +   * @param sc - Your SparkContext.
    +   * @param schema - The Catalyst schema of the underlying database table.
    +   * @param driver - The class name of the JDBC driver for the given url.
    +   * @param url - The JDBC url to connect to.
    +   * @param fqTable - The fully-qualified table name (or paren'd SQL 
query) to use.
    +   * @param requiredColumns - The names of the columns to SELECT.
    +   * @param filters - The filters to include in all WHERE clauses.
    +   * @param parts - An array of JDBCPartitions specifying partition ids and
    +   *    per-partition WHERE clauses.
    +   *
    +   * @return An RDD representing "SELECT requiredColumns FROM fqTable".
    +   */
    +  def scanTable(sc: SparkContext,
    --- End diff --
    
    just a quick note - the spark coding style does 4 space indent for function 
parameters in definitions. e.g.
    ```scala
    def scanTable(
        sc: SparkContext,
        schema: StructType,
        ...): RDD[Row => {
    ```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at [email protected] or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to