You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by budde <gi...@git.apache.org> on 2017/02/15 22:12:23 UTC

[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

GitHub user budde opened a pull request:

    https://github.com/apache/spark/pull/16944

    [SPARK-19611][SQL] Introduce configurable table schema inference

    *Update: Accidentally broke #16942 via a force push. Opening a replacement PR.*
    
    Replaces #16797. See the discussion in this PR for more details/justification for this change.
    
    ## Summary of changes
    [JIRA for this change](https://issues.apache.org/jira/browse/SPARK-19611)
    
    - Add spark.sql.hive.schemaInferenceMode param to SQLConf
    - Add schemaFromTableProps field to CatalogTable (set to true when schema is
      successfully read from table props)
    - Perform schema inference in HiveMetastoreCatalog if schemaFromTableProps is
      false, depending on spark.sql.hive.schemaInferenceMode.
    - Update table metadata properties in HiveExternalCatalog.alterTable()
    - Add HiveSchemaInferenceSuite tests
    
    ## How was this patch tested?
    
    The tests in HiveSchemaInferenceSuite should verify that schema inference is working as expected.
    
    ## Open issues
    
    - The option values for ```spark.sql.hive.schemaInferenceMode ``` (e.g. "INFER_AND_SAVE", "INFER_ONLY", "NEVER_INFER") should be made into constants or an enum. I couldn't find a sensible object to place them in though that doesn't introduce a dependency between sql/core and sql/hive.
    - Should "INFER_AND_SAVE" be the default behavior? This restores the out-of-the-box compatibility that was present prior to 2.1.0 but changes the behavior of 2.1.0 (which is essentially "NEVER_INFER").
    - Is ```HiveExternalCatalog.alterTable()``` the appropriate place to write back the table metadata properties outside of createTable()? Should a new external catalog method like updateTableMetadata() be introduced?
    - All partition columns will still be treated as case-insensitive even after inferring. As far as I remember, this has always been the case with schema inference prior to Spark 2.1.0 and I haven't made any attempts to reconcile this since it doesn't cause the same problems that case sensitive data fields do. Should we attempt to restore case sensitivity by inspecting file paths or leave this as-is?

You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/budde/spark SPARK-19611

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/spark/pull/16944.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #16944
    
----

----


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Thanks for the thorough feedback.
    
    In the latest update I refactored ```convertToLogicalRelation()``` to update the schema of the ```catalogTable``` field of the value it returns whenever a schema is successfully inferred rather than only when one is inferred and successfully saved. This allowed me to clean up the changes to this method a bit. @ericl -- feel free to give this another look over.
    
    Per @cloud-fan's feedback, I added an ```alterTableSchema()``` method to the ExternalTable interface. I've added implementations of it to ```InMemoryCatalog``` and ```HiveExternalCatalog``` along with tests. I've also addressed the feedback from @viirya 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104598287
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -476,46 +476,6 @@ object ParquetFileFormat extends Logging {
       }
     
       /**
    -   * Reconciles Hive Metastore case insensitivity issue and data type conflicts between Metastore
    -   * schema and Parquet schema.
    -   *
    -   * Hive doesn't retain case information, while Parquet is case sensitive. On the other hand, the
    -   * schema read from Parquet files may be incomplete (e.g. older versions of Parquet doesn't
    -   * distinguish binary and string).  This method generates a correct schema by merging Metastore
    -   * schema data types and Parquet schema field names.
    -   */
    -  def mergeMetastoreParquetSchema(
    -      metastoreSchema: StructType,
    -      parquetSchema: StructType): StructType = {
    -    def schemaConflictMessage: String =
    -      s"""Converting Hive Metastore Parquet, but detected conflicting schemas. Metastore schema:
    -         |${metastoreSchema.prettyJson}
    -         |
    -         |Parquet schema:
    -         |${parquetSchema.prettyJson}
    -       """.stripMargin
    -
    -    val mergedParquetSchema = mergeMissingNullableFields(metastoreSchema, parquetSchema)
    --- End diff --
    
    No other places use `mergeMissingNullableFields` anymore. If we remove `mergeMetastoreParquetSchema`, `mergeMissingNullableFields` can be removed too.



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104759435
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{
    +  Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      tableName: String,
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      hiveExternalCatalogTable(
    +        tableName,
    +        serde,
    +        dir.getAbsolutePath,
    +        lowerCaseSchema(schema),
    +        partitionCols.map(_.toLowerCase)),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      val partitions = (0 until NUM_RECORDS).map { idx =>
    +         hiveCatalogPartition(dir.getAbsolutePath, serde, partitionCols, idx)
    +      }.toSeq
    +      client.createPartitions(DATABASE, tableName, partitions, true)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, tableName).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, tableName)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (String, Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val tableName = "test_table"
    +        val schema = setupExternalTable(tableName, fileType, fields, partCols, dir)
    +        try f(
    --- End diff --
    
    use `withTable`, which can drop the table at last


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73858/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    After `hive` alteration, I run `spark-shell` and set the following immediately.
    But, it's already broken. When do you save the new inferred schema?
    ```
    scala> sql("set spark.sql.hive.caseSensitiveInferenceMode=INFER_ONLY").show
    +--------------------+----------+
    |                 key|     value|
    +--------------------+----------+
    |spark.sql.hive.ca...|INFER_ONLY|
    +--------------------+----------+
    
    
    scala> sql("SELECT a, b FROM t3").show
    +---+----+
    |  a|   b|
    +---+----+
    |100|null|
    +---+----+
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74076/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Oh, whenever I starts `spark-shell`, I should do the following, or may have that in Spark configuration. Otherwise, it shows wrong result.
    ```
    sql("set spark.sql.hive.caseSensitiveInferenceMode=NEVER_INFER").show
    ```
    It looks like `NEVER_INFER` is the only safe default value for this regression.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104243616
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -597,6 +597,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
         }
       }
     
    +  override def alterTableSchema(db: String, table: String, schema: StructType): Unit = withClient {
    +    requireTableExists(db, table)
    +    val rawTable = getRawTable(db, table)
    +    val withNewSchema = rawTable.copy(schema = schema)
    +    // Add table metadata such as table schema, partition columns, etc. to table properties.
    +    val updatedTable = withNewSchema.copy(
    +      properties = withNewSchema.properties ++ tableMetaToTableProps(withNewSchema))
    +    client.alterTable(updatedTable)
    --- End diff --
    
    one more thing, if you look at `def createDataSourceTable`, we need to care about one special case: when saving table to hive, we should try catch it, if it fails, convert it to non-hive-compatible format(set schema to Nil) and save again. We should follow it here, in case the schema we wanna alter to is not hive compatible.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104249293
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +235,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    --- End diff --
    
    This was used in the previous code for schema inference. I think the only reason this would still be needed is if the metastore schema contains a nullable field that isn't actually present in the underlying Parquet data. See #5214 for more details here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101908155
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    --- End diff --
    
    Good call, I'll add a check for ```schemaPreservesCase```.
    
    We'll have to use the underlying HiveClient to obtain the raw table in order to check the presence of ```DATASOURCE_SCHEMA_NUMPARTS``` instead of ```originalTable``` directly since HiveExternalCatalog [filters out any property starting with SPARK_SQL_PREFIX](https://github.com/apache/spark/blob/f48c5a57d6488d5598534ca5834e008504f464fe/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala#L665-L668).
    
    I'm thinking of just adding these checks to the ```setupCaseSensitiveTable()``` method since we're essentially just asserting that our initial conditions are what we expect (table returned by catalog has schemaPreservesCase=false and the underlying table contains no Spark properties).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #72959 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72959/testReport)** for PR 16944 at commit [`0bca163`](https://github.com/apache/spark/commit/0bca163a2ad53a93e629874a3cc0275532b25bac).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104751264
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala ---
    @@ -71,4 +72,19 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite {
         assert(!rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_PROVIDER))
         assert(DDLUtils.isHiveTable(externalCatalog.getTable("db1", "hive_tbl")))
       }
    +
    +  test("alter table schema should save case-sensitive schema to properties") {
    --- End diff --
    
    `HiveExternalCatalogSuite` extends `ExternalCatalogSuite`, so https://github.com/apache/spark/pull/16944/files#diff-8c4108666a6639034f0ddbfa075bcb37R243 will also be run here. Do we still need this test?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104751915
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala ---
    @@ -71,4 +72,19 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite {
         assert(!rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_PROVIDER))
         assert(DDLUtils.isHiveTable(externalCatalog.getTable("db1", "hive_tbl")))
       }
    +
    +  test("alter table schema should save case-sensitive schema to properties") {
    --- End diff --
    
    This test checks that the case-sensitive schema is written to Hive as table properties when using ```HiveExternalCatalog```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104085920
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -195,17 +207,20 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               fileFormatClass,
               None)
             val logicalRelation = cached.getOrElse {
    +          val fileIndex = new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    --- End diff --
    
    I was thinking that ```InMemoryFileIndex``` had similar semantics to ```CatalogFileIndex``` where the leaf files are only listed if  ```listFiles()``` is invoked. I can make the ```fileIndex``` arg of ```inferIfNeeded()``` a ```Option[FileIndex]``` and create the ```InMemoryFileIndex``` within that method instead if we need to infer a schema and no file index is provided.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73444/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102550255
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,21 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    +    val INFER_AND_SAVE, INFER_ONLY, NEVER_INFER = Value
    +  }
    +  val HIVE_CASE_SENSITIVE_INFERENCE = buildConf("spark.sql.hive.caseSensitiveInferenceMode")
    +    .doc("Configures the action to take when a case-sensitive schema cannot be read from a Hive " +
    +      "table's properties. Valid options include INFER_AND_SAVE (infer the case-sensitive " +
    --- End diff --
    
    Probably want to add some explanation as to why this matters even though SQL is not case sensitive, e.g.
    // Note that Spark SQL itself not case-sensitive, but since file formats such as parquet are, a case-sensitive schema is required in order to properly read fields.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103049597
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,51 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    --- End diff --
    
    I think `dataSchema` is a better name, why change it?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102547965
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,21 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    --- End diff --
    
    I'm trying to avoid using string literals. If we want to change the possible values for this param we would need to find each and every place the literal value is used and update it. I think this is too flaky and runs the risk of introducing bugs that will only be apparent at runtime. Expressing this as an enumeration gives us some level of type safety and at the very least will cause a compiler error if the possible values are changed and comparisons elsewhere in the code aren't updated.
    
    I'm willing to remove the enumeration if it isn't consistent with Spark code practices but at the very least the possible values should be expressed as constants rather than literals.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103049915
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -226,6 +258,41 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
       }
     
    +  private def inferSchema(
    +      metastoreSchema: StructType,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileType: String,
    +      fileIndex: FileIndex): Option[StructType] = {
    +    val inferred = fileFormat.inferSchema(
    +      sparkSession,
    +      options,
    +      fileIndex.listFiles(Nil).flatMap(_.files))
    +    if (fileType.equals("parquet")) {
    +      inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +    } else {
    +      inferred
    +    }
    +  }
    +
    +  private def updateCatalogTable(
    +      catalogTable: CatalogTable,
    +      inferredSchema: Option[StructType]): Option[CatalogTable] = try {
    +    inferredSchema.flatMap { schema =>
    --- End diff --
    
    nit: `foreach`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73497 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73497/testReport)** for PR 16944 at commit [`b8d72b4`](https://github.com/apache/spark/commit/b8d72b467e01880c0870a553e27dd0ac771e5489).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104271004
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    +          case _ =>
    +            schema
    +        }
    +      }
    +
    +      inferredSchema match {
    +        case Some(schema) =>
    +          if (inferenceMode == INFER_AND_SAVE) {
    +            updateCatalogSchema(relation.tableMeta.identifier, schema)
    --- End diff --
    
    Ah, I see. So we need to be merging the partition fields back into the table schema since that's what the metastore expects. I'll fix the code.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103048200
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -510,8 +510,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
         requireTableExists(db, tableDefinition.identifier.table)
         verifyTableProperties(tableDefinition)
     
    +    // Add table metadata such as table schema, partition columns, etc. if they aren't already
    +    // present.
    +    val withMetaProps = tableDefinition.copy(
    --- End diff --
    
    `alterTable` assumes we won't alter table schema(see the doc of `alterTable`), and I'm not sure if we can just put the schema into table properties and alter it.
    
    I'd like to create a new method `def alterSchema(db: String, table: String, schema: StructType)`, which reads the raw hive table, put table schema into table properties and alter it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103049318
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,25 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    +    val INFER_AND_SAVE, INFER_ONLY, NEVER_INFER = Value
    +  }
    +
    +  val HIVE_CASE_SENSITIVE_INFERENCE = buildConf("spark.sql.hive.caseSensitiveInferenceMode")
    +    .doc("Sets the action to take when a case-sensitive schema cannot be read from a Hive " +
    +      "table's properties. Although Spark SQL itself is not case-sensitive, Hive compatible file " +
    +      "formats such as Parquet are. Spark SQL must use a case-preserving schema when querying " +
    +      "any table backed by files containing case-sensitive field names or queries may not return " +
    +      "accurate results. Valid options include INFER_AND_SAVE (infer the case-sensitive " +
    --- End diff --
    
    we should also mention that it's the default one


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102554021
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,70 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +          val inferredSchema = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode != HiveCaseSensitiveInferenceMode.NEVER_INFER) {
    +            logInfo(s"Inferring case-sensitive schema for table $tableIdentifier")
    --- End diff --
    
    Will do


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101461357
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  // Create a CatalogTable instance modeling an external Hive table in a metastore that isn't
    +  // controlled by Spark (i.e. has no Spark-specific table properties set).
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option("default")),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val lowercaseSchema = StructType(Seq(
    +      StructField("fieldone", LongType),
    +      StructField("partcol1", IntegerType),
    +      StructField("partcol2", IntegerType)))
    +
    +    val client = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog].client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions("default", tableName, partitions, true)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  test("Queries against case-sensitive tables with no schema in table properties should work " +
    +    "when schema inference is enabled") {
    +    withSQLConf("spark.sql.hive.schemaInferenceMode" -> "INFER_AND_SAVE") {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val expectedSchema = StructType(Seq(
    +            StructField("fieldOne", LongType),
    +            // Partition columns remain case-insensitive
    +            StructField("partcol1", IntegerType),
    +            StructField("partcol2", IntegerType)))
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          // Test that the case-sensitive schema was storied as a table property after inference
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == expectedSchema)
    --- End diff --
    
    I'll change both of these tests to check the catalog table as well


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104086046
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -597,6 +597,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
         }
       }
     
    +  override def alterTableSchema(db: String, table: String, schema: StructType): Unit = withClient {
    +    requireTableExists(db, table)
    +    val rawTable = getRawTable(db, table)
    +    val withNewSchema = rawTable.copy(schema = schema)
    --- End diff --
    
    At line 606 we need to invoke ```tableMetaToTableProps()``` with a ```CatalogTable``` record containing the new table schema in order to generate the Hive table properties based on the new schema.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @dongjoon-hyun This problem is happening because [```HiveMetastoreCatalog.mergeWithMetastoreSchema()```](https://github.com/apache/spark/blob/master/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala#L350-L376) doesn't maintain the ```StructType``` field order returned by the metastore. I overlooked that the ordering of the schema must be maintained. I'll open a new PR to correct this and add tests where appropriate.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104245300
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -195,17 +211,19 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               fileFormatClass,
               None)
             val logicalRelation = cached.getOrElse {
    +          val (dataSchema, updatedTable)
    +            = inferIfNeeded(relation, options, metastoreSchema, fileFormat)
    --- End diff --
    
    nit:
    ```
    val (a, b) = 
      someLongFunctionCall(...)
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73858 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73858/testReport)** for PR 16944 at commit [`d0652f9`](https://github.com/apache/spark/commit/d0652f93dd41ed91e753dabc755788f9a9b7f2c6).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74133/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @cloud-fan 


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103046982
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,25 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    --- End diff --
    
    is this type alias used?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74196 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74196/testReport)** for PR 16944 at commit [`8e1e2aa`](https://github.com/apache/spark/commit/8e1e2aa366f9b77242656331279410e637e727cd).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104757327
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{
    +  Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    --- End diff --
    
    this method only be called once, so the `tableName`, `fileType`, etc. are fixed. Then why do we have these parameters? can we just hard code these values?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104750372
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +229,55 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.unquotedString
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(relation.tableMeta.location)
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = fileFormat
    +        .inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        .map(mergeWithMetastoreSchema(relation.tableMeta.schema, _))
    +
    +      inferredSchema match {
    +        case Some(schema) =>
    +          if (inferenceMode == INFER_AND_SAVE) {
    +            updateCatalogSchema(relation.tableMeta.identifier, schema)
    +          }
    +          (schema, relation.tableMeta.copy(schema = schema))
    +        case None =>
    +          logWarning(s"Unable to infer schema for table $tableName from file format " +
    +            s"$fileFormat (inference mode: $inferenceMode). Using metastore schema.")
    +          (relation.tableMeta.schema, relation.tableMeta)
    +      }
    +    } else {
    +      (relation.tableMeta.schema, relation.tableMeta)
    +    }
    +  }
    +
    +  private def updateCatalogSchema(identifier: TableIdentifier, schema: StructType): Unit = try {
    +    val db = identifier.database.get
    +    val table = identifier.table
    +    logInfo(s"Saving case-sensitive schema for table $table")
    --- End diff --
    
    nit: use `identifier. unquotedString`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    I've updated the PR based on feedback received. Changes from previous commit:
    
    - Fixed a couple indent issues
    - Clarify some HiveSchemaInferenceSuite comments and general cleanup
    - Add CatalogTable checks and ```NEVER_INFER``` test to SchemaInferenceSuite
    - Added additional info/error logging to HiveMetastoreCatalog
    - Catch nonfatal exceptions from alterTable() call in HiveMetastoreCatalog
    - Change param name to ```spark.sql.hive.caseSensitiveInferenceMode```
    - Rename ```CatalogTable.schemaFromTableProps``` to ```CatalogTable.schemaPreservesCase```
    - Introduce ```HiveCaseSensitiveInferenceMode``` enumeration type to get rid of "magic strings" used for inference modes
    - Use updated CatalogTable record in LogicalRelation if ```INFER_AND_SAVE``` used


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101625724
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,21 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    --- End diff --
    
    Is there a more-appropriate place I can put this Enumeration?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Should the option be set before `CREATE TABLE`? If then, it seems that we cannot prevent the corruption of the existing parquet tables.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73828 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73828/testReport)** for PR 16944 at commit [`95af481`](https://github.com/apache/spark/commit/95af4810b9c85b2b8680d7791cf298ed147e33c6).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104245195
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -195,17 +211,19 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               fileFormatClass,
               None)
             val logicalRelation = cached.getOrElse {
    +          val (dataSchema, updatedTable)
    +            = inferIfNeeded(relation, options, metastoreSchema, fileFormat)
               val created =
                 LogicalRelation(
                   DataSource(
                     sparkSession = sparkSession,
                     paths = rootPath.toString :: Nil,
    -                userSpecifiedSchema = Some(metastoreSchema),
    +                userSpecifiedSchema = Option(dataSchema),
                     // We don't support hive bucketed tables, only ones we write out.
                     bucketSpec = None,
                     options = options,
                     className = fileType).resolveRelation(),
    -              catalogTable = Some(relation.tableMeta))
    +               catalogTable = Option(updatedTable))
    --- End diff --
    
    the previous indent is corrected.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101456600
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,23 +161,49 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    +          val filteredMetastoreSchema = StructType(metastoreSchema
                   .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
     
    +          val inferenceMode = sparkSession.sessionState.conf.schemaInferenceMode
    +          val dataSchema = if (inferenceMode != "NEVER_INFER" &&
    +              !catalogTable.schemaFromTableProps) {
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    --- End diff --
    
    Can we log info that we are going to infer schema (and save it to metastore)?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104081425
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -597,6 +597,16 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
         }
       }
     
    +  override def alterTableSchema(db: String, table: String, schema: StructType): Unit = withClient {
    +    requireTableExists(db, table)
    +    val rawTable = getRawTable(db, table)
    +    val withNewSchema = rawTable.copy(schema = schema)
    --- End diff --
    
    Why we do two `copy`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104750390
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -124,11 +130,14 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
           fileFormatClass: Class[_ <: FileFormat],
           fileType: String): LogicalRelation = {
         val metastoreSchema = relation.tableMeta.schema
    -    val tableIdentifier =
    -      QualifiedTableName(relation.tableMeta.database, relation.tableMeta.identifier.table)
    +    val tableIdentifier = QualifiedTableName(
    +      relation.tableMeta.database, relation.tableMeta.identifier.table)
    +    val bucketSpec = None  // We don't support hive bucketed tables, only ones we write out.
    --- End diff --
    
    Looks like I didn't merge an upstream change properly. My bad. Will fix


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Ur, I've got the following.
    
    ```
    scala> sql("set spark.sql.hive.caseSensitiveInferenceMode=INFER_ONLY").show
    +--------------------+----------+
    |                 key|     value|
    +--------------------+----------+
    |spark.sql.hive.ca...|INFER_ONLY|
    +--------------------+----------+
    
    scala> sql("SELECT a, b FROM t1").show
    +---+----+
    |  a|   b|
    +---+----+
    |100|null|
    +---+----+
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Pinging @viirya and @ericl to take a look at the updates per their feedback


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74154/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Looks like I missed a Catalyst test. Updating the PR.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74149 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74149/testReport)** for PR 16944 at commit [`af92b0c`](https://github.com/apache/spark/commit/af92b0c678f950341411042f783f39fa5ec4bc47).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104268303
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    --- End diff --
    
    Good call, I'll remove this argument


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104268134
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -597,6 +597,27 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
         }
       }
     
    +  override def alterTableSchema(db: String, table: String, schema: StructType): Unit = withClient {
    +    requireTableExists(db, table)
    +    val rawTable = getRawTable(db, table)
    +    val withNewSchema = rawTable.copy(schema = schema)
    +    // Add table metadata such as table schema, partition columns, etc. to table properties.
    +    val updatedTable = withNewSchema.copy(
    +      properties = withNewSchema.properties ++ tableMetaToTableProps(withNewSchema))
    +    try {
    +      client.alterTable(updatedTable)
    +    } catch {
    +      case NonFatal(e) =>
    +        val warningMessage =
    +          s"Could not alter schema of table  ${rawTable.identifier.quotedString} in a Hive " +
    +            "compatible way. Updating Hive metastore in Spark SQL specific format."
    +        logWarning(warningMessage, e)
    +        client.alterTable(updatedTable.copy(
    +          storage = updatedTable.storage.copy(locationUri = None),
    --- End diff --
    
    Will remove


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101461282
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -163,6 +163,10 @@ case class BucketSpec(
      * @param tracksPartitionsInCatalog whether this table's partition metadata is stored in the
      *                                  catalog. If false, it is inferred automatically based on file
      *                                  structure.
    + * @param schemaFromTableProps Whether the schema field was obtained by parsing a case-sensitive
    --- End diff --
    
    Since table props is an implementation detail, consider naming this `schemaPreservesCase`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Re-pinging participants from #16797: @gatorsmile, @viirya, @ericl, @mallman and @cloud-fan. Sorry for the noise.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73123 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73123/testReport)** for PR 16944 at commit [`980311f`](https://github.com/apache/spark/commit/980311f3382a4588326736c8b83d449905b74c92).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102369195
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,21 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    --- End diff --
    
    we can follow `PARQUET_COMPRESSION` and write the string literal directly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Pinging participants from #16797 once more to get any feedback on the new proposal: @gatorsmile, @viirya, @ericl, @mallman and @cloud-fan


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72978/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101461106
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,17 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  val HIVE_SCHEMA_INFERENCE_MODE = buildConf("spark.sql.hive.schemaInferenceMode")
    +    .doc("Configures the action to take when a case-sensitive schema cannot be read from a Hive " +
    +      "table's properties. Valid options include INFER_AND_SAVE (infer the case-sensitive " +
    +      "schema from the underlying data files and write it back to the table properties), " +
    +      "INFER_ONLY (infer the schema but don't attempt to write it to the table properties) and " +
    +      "NEVER_INFER (fallback to using the case-insensitive metastore schema instead of inferring).")
    +    .stringConf
    +    .transform(_.toUpperCase())
    +    .checkValues(Set("INFER_AND_SAVE", "INFER_ONLY", "NEVER_INFER"))
    +    .createWithDefault("INFER_AND_SAVE")
    --- End diff --
    
    INFER_ONLY seems better to me as a default since it avoid throwing exceptions. To avoid silent performance degradation in this case, perhaps we can log a warning pointing to this config? Alternatively, I could see INFER_AND_SAVE as the default if we caught the write permission errors.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101460842
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,23 +161,49 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    +          val filteredMetastoreSchema = StructType(metastoreSchema
                   .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    --- End diff --
    
    I'll fix both of these


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104761129
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{
    +  Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      tableName: String,
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      hiveExternalCatalogTable(
    +        tableName,
    +        serde,
    +        dir.getAbsolutePath,
    +        lowerCaseSchema(schema),
    +        partitionCols.map(_.toLowerCase)),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      val partitions = (0 until NUM_RECORDS).map { idx =>
    +         hiveCatalogPartition(dir.getAbsolutePath, serde, partitionCols, idx)
    +      }.toSeq
    +      client.createPartitions(DATABASE, tableName, partitions, true)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, tableName).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, tableName)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (String, Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val tableName = "test_table"
    +        val schema = setupExternalTable(tableName, fileType, fields, partCols, dir)
    +        try f(
    +          tableName,
    +          fields,
    +          partCols,
    +          schema) finally spark.sql(s"DROP TABLE IF EXISTS $tableName")
    +      }
    +    }
    +  }
    +
    +  private def withFileTypes(f: (String) => Unit): Unit
    --- End diff --
    
    I didn't do this since I wanted to be able to wrap the  ```test("...") { ... }``` block in ```withFileTypes { ... }``` in order to have individual tests per file type. The thought here is that this will help identify if schema inference is broken just for a particular file format.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #72978 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72978/testReport)** for PR 16944 at commit [`8ac3b04`](https://github.com/apache/spark/commit/8ac3b04653c29c91d727f51906fa7dd51c2d08b8).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74133 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74133/testReport)** for PR 16944 at commit [`aa433ec`](https://github.com/apache/spark/commit/aa433ec794c156c513897001427ccf260a1c3479).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Maybe, `t1` is already corrupted. Let me try a new one with that option.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101888190
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    --- End diff --
    
    Could we have a check for `originalTable` like below to see if its `schemaPreservesCase` is false and there is no `DATASOURCE_SCHEMA_NUMPARTS` property?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74155 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74155/testReport)** for PR 16944 at commit [`8e1e2aa`](https://github.com/apache/spark/commit/8e1e2aa366f9b77242656331279410e637e727cd).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101460565
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,17 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  val HIVE_SCHEMA_INFERENCE_MODE = buildConf("spark.sql.hive.schemaInferenceMode")
    +    .doc("Configures the action to take when a case-sensitive schema cannot be read from a Hive " +
    +      "table's properties. Valid options include INFER_AND_SAVE (infer the case-sensitive " +
    +      "schema from the underlying data files and write it back to the table properties), " +
    +      "INFER_ONLY (infer the schema but don't attempt to write it to the table properties) and " +
    +      "NEVER_INFER (fallback to using the case-insensitive metastore schema instead of inferring).")
    +    .stringConf
    +    .transform(_.toUpperCase())
    +    .checkValues(Set("INFER_AND_SAVE", "INFER_ONLY", "NEVER_INFER"))
    +    .createWithDefault("INFER_AND_SAVE")
    --- End diff --
    
    This was proposed in #16797 but I'd like to open this for discussion.
    - ```INFER_ONLY``` would mimic the pre-2.1.0 behavior.
    - ```INFER_AND_SAVE``` would attempt to prevent future inferences but may fail if the Hive client doesn't have write permissions on the metastore. 
    - ```NEVER_INFER``` is the current behavior in 2.1.0 which breaks support with the tables affected by [SPARK-19611](https://issues.apache.org/jira/browse/SPARK-19611). Users may wish to enable this mode for tables without the table properties schema that they know are case-insensitive.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73828/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104262277
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -169,16 +179,22 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
                 }
               }
     
    +          val (dataSchema, updatedTable) = inferIfNeeded(
    +            relation,
    +            options,
    +            filteredMetastoreSchema,
    +            fileFormat,
    +            Option(fileIndex))
    +
               val fsRelation = HadoopFsRelation(
                 location = fileIndex,
                 partitionSchema = partitionSchema,
    -            dataSchema = relation.tableMeta.dataSchema,
    +            dataSchema = dataSchema,
                 // We don't support hive bucketed tables, only ones we write out.
                 bucketSpec = None,
    -            fileFormat = fileFormatClass.newInstance(),
    +            fileFormat = fileFormat,
                 options = options)(sparkSession = sparkSession)
    -
    -          val created = LogicalRelation(fsRelation, catalogTable = Some(relation.tableMeta))
    +          val created = LogicalRelation(fsRelation, catalogTable = Option(updatedTable))
    --- End diff --
    
    yea, "let it fail and fix the bug" is better


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73720/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74131/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101456890
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,23 +161,49 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    +          val filteredMetastoreSchema = StructType(metastoreSchema
                   .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
     
    +          val inferenceMode = sparkSession.sessionState.conf.schemaInferenceMode
    +          val dataSchema = if (inferenceMode != "NEVER_INFER" &&
    +              !catalogTable.schemaFromTableProps) {
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    +            val merged = if (fileType.equals("parquet")) {
    +              inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +            } else {
    +              inferred
    +            }
    +            if (inferenceMode == "INFER_AND_SAVE") {
    +              // If a case-sensitive schema was successfully inferred, execute an alterTable
    +              // operation to save the schema to the table properties.
    +              merged.foreach { mergedSchema =>
    +                  val updatedTable = catalogTable.copy(schema = mergedSchema)
    --- End diff --
    
    Wrong indent style.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Thank you for quick investigation. Yep. please go ahead!
    
    BTW, can we hold on backporting (#17229) for a while before resolving all issues?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74051 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74051/testReport)** for PR 16944 at commit [`1e3af23`](https://github.com/apache/spark/commit/1e3af2355bce19c55e797b8d3ef139eba6da944a).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103068442
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    --- End diff --
    
    nit: simplified test title like `Schema should be inferred and saved when INFER_AND_SAVE is specified`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103116517
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    --- End diff --
    
    Fixed the typo. Thanks


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103068578
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          // Test that the case-sensitive schema was storied as a table property after inference
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == caseSensitiveSchema)
    +
    +          // Verify the catalog table now contains the udpated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(catalogTable.schemaPreservesCase == true)
    +          assert(catalogTable.schema == caseSensitiveSchema)
    +          assert(catalogTable.partitionColumnNames == Seq("partcol1", "partcol2"))
    +          val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_SCHEMA_NUMPARTS))
    +        }
    +      }
    +    }
    +  }
    +
    +  test("Schema should be inferred but not stored when INFER_ONLY is specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_ONLY.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          val existingSchema = spark.sql(SELECT_ALL_QUERY).schema
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == existingSchema)
    +          // Catalog table shouldn't be altered
    +          assert(externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) == originalTable)
    +        }
    +      }
    +    }
    +  }
    +
    +  test("Schema should not be inferred when NEVER_INFER is specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.NEVER_INFER.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          // Only check the schema returned by SELECT * as other queries will break
    --- End diff --
    
    Then we can test if we get 0 rows back?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74026/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102850496
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,25 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    +    val INFER_AND_SAVE, INFER_ONLY, NEVER_INFER = Value
    +  }
    +
    +  // Although Spark SQL itself is not case-sensitive, some of the underlying data storage formats
    +  // it supports such as Parquet are. Spark must use the correct schema when querying against data
    +  // files containing case-sensitive names or field values will not be resolved properly.
    --- End diff --
    
    Gotcha. Will fix.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r105268502
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.DATASOURCE_SCHEMA_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType = {
    +    StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      CatalogTable(
    +        identifier = TableIdentifier(table = TEST_TABLE_NAME, database = Option(DATABASE)),
    +        tableType = CatalogTableType.EXTERNAL,
    +        storage = CatalogStorageFormat(
    +          locationUri = Option(new java.net.URI(dir.getAbsolutePath)),
    +          inputFormat = serde.inputFormat,
    +          outputFormat = serde.outputFormat,
    +          serde = serde.serde,
    +          compressed = false,
    +          properties = Map("serialization.format" -> "1")),
    +        schema = schema,
    +        provider = Option("hive"),
    +        partitionColumnNames = partitionCols.map(_.toLowerCase),
    +        properties = Map.empty),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      spark.catalog.recoverPartitions(TEST_TABLE_NAME)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val schema = setupExternalTable(fileType, fields, partCols, dir)
    +        withTable(TEST_TABLE_NAME) { f(fields, partCols, schema) }
    +      }
    +    }
    +  }
    +
    +  private def withFileTypes(f: (String) => Unit): Unit
    +    = Seq(ORC_FILE_TYPE, PARQUET_FILE_TYPE).foreach(f)
    +
    +  private def withInferenceMode(mode: InferenceMode)(f: => Unit): Unit = {
    +    withSQLConf(
    +      HiveUtils.CONVERT_METASTORE_ORC.key -> "true",
    +      SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key -> mode.toString)(f)
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  private def testFieldQuery(fields: Seq[String]): Unit = {
    +    if (!fields.isEmpty) {
    +      val query = s"SELECT * FROM ${TEST_TABLE_NAME} WHERE ${Random.shuffle(fields).head} >= 0"
    +      assert(spark.sql(query).count == NUM_RECORDS)
    +    }
    +  }
    +
    +  private def testTableSchema(expectedSchema: StructType): Unit
    +    = assert(spark.table(TEST_TABLE_NAME).schema == expectedSchema)
    +
    +  withFileTypes { fileType =>
    +    test(s"$fileType: schema should be inferred and saved when INFER_AND_SAVE is specified") {
    +      withInferenceMode(INFER_AND_SAVE) {
    +        withTestTables(fileType) { (fields, partCols, schema) =>
    +          testFieldQuery(fields)
    +          testFieldQuery(partCols)
    +          testTableSchema(schema)
    +
    +          // Verify the catalog table now contains the updated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(catalogTable.schemaPreservesCase)
    +          assert(catalogTable.schema == schema)
    +          assert(catalogTable.partitionColumnNames == partCols.map(_.toLowerCase))
    --- End diff --
    
    one follow-up: we can also infer the partition columns and make them case sensitive.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104796484
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala ---
    @@ -71,4 +72,19 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite {
         assert(!rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_PROVIDER))
         assert(DDLUtils.isHiveTable(externalCatalog.getTable("db1", "hive_tbl")))
       }
    +
    +  test("alter table schema should save case-sensitive schema to properties") {
    --- End diff --
    
    Fair enough. My only concern here would be that ```alterTableSchema()``` is using a different code path to set the Hive table properties than ```createTable()``` does.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101461155
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,162 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  // Create a CatalogTable instance modeling an external Hive table in a metastore that isn't
    +  // controlled by Spark (i.e. has no Spark-specific table properties set).
    --- End diff --
    
    I wrote the method to take arbitrary properties but for the purposes of this test only an empty map is supplied. I'll make the comment more applicable to the method though and describe the usage of it elsewhere.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74051/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73808/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @dongjoon-hyun Hmm, we should be catching and logging this in [this catch block](https://github.com/apache/spark/blob/f79371ad86d94da14bd1ddb53e99a388017b6892/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala#L275-L276). How exactly are you running this? Are you just using ```spark-shell```?
    
    Can you also verify that no exception is thrown when the ```spark.sql.hive.caseSensitiveInferenceMode``` SQL param is set to ```INFER_ONLY```?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74083/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104250125
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -159,6 +165,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               Some(partitionSchema))
     
             val logicalRelation = cached.getOrElse {
    +          val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    --- End diff --
    
    `relation.tableMeta.dataSchema` will filter out partition columns


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104750878
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -286,3 +347,33 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         }
       }
     }
    +
    +private[hive] object HiveMetastoreCatalog {
    --- End diff --
    
    Good call, should've caught that. Will fix


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    I'm looking into it. I would suspect altering the schema outside of Spark combined with using a Spark-managed metastore rather than an external metastore may be hitting some sort of edge case. I have everything set up locally but ```beeline``` can't seem to find any of the Hive jars so I'll have to look into that.
    
    It would be possible to make ```NEVER_INFER``` the default option instead of ```INFER_AND_SAVE``` but this will result in queries that silently break for any external Hive Metastore table that contains a case-sensitive field name in its underlying data. I want to identify the particular behavior that's happening here and see if there's a suitable way to fix it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103047268
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala ---
    @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path
     
     import org.apache.spark.sql._
     import org.apache.spark.sql.execution.WholeStageCodegenExec
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    --- End diff --
    
    useless import?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104796641
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,341 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = TEST_TABLE_NAME, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      hiveExternalCatalogTable(
    +        serde,
    +        dir.getAbsolutePath,
    +        lowerCaseSchema(schema),
    +        partitionCols.map(_.toLowerCase)),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      val partitions = (0 until NUM_RECORDS).map { idx =>
    --- End diff --
    
    how about we just call `spark.catalog.recoverPartitions(TEST_TABLE_NAME)` here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73725/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102548681
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -690,10 +696,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
               "different from the schema when this table was created by Spark SQL" +
               s"(${schemaFromTableProps.simpleString}). We have to fall back to the table schema " +
               "from Hive metastore which is not case preserving.")
    -        hiveTable
    +        hiveTable.copy(schemaPreservesCase = false)
    --- End diff --
    
    In this case we are discarding the schema obtained from the table properties and explicitly falling back to using the case-insenstive schema obtained from the metastore. ```schemaPreservesCase``` needs to be set to ```false``` here for the same reason it does at [line 702](https://github.com/apache/spark/pull/16944/files/15c25e0fc23ec48ecd0fad7ca7c60d3a82d32a73#diff-159191585e10542f013cb3a714f26075R702).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74076 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74076/testReport)** for PR 16944 at commit [`9bf6a3a`](https://github.com/apache/spark/commit/9bf6a3a75d9a9f287d5b05eab4071c0c3e9db267).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103067980
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,51 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +          val shouldInfer = (inferenceMode != NEVER_INFER) && !catalogTable.schemaPreservesCase
    +
    +          val (dataSchema, updatedTable) = if (shouldInfer) {
    +            logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +              s"$inferenceMode)")
    +            val inferredSchema
    +              = inferSchema(metastoreSchema, options, defaultSource, fileType, fileIndex)
    +
    +            // If configured, save the inferred case-sensitive schema to the table properties and
    +            // fetch the updated CatalogTable record for use in the LogicalRelation.
    +            val updatedTable = if (inferenceMode == INFER_AND_SAVE) {
    +              updateCatalogTable(catalogTable, inferredSchema).getOrElse {
    +                logWarning(s"Unable to save case-sensitive schema for table $tableName " +
    +                  s"(inference mode: $inferenceMode)")
    +                catalogTable
    +              }
    +            } else {
    +              catalogTable
    +            }
    +            val schema = inferredSchema.getOrElse {
    --- End diff --
    
    Should we put this before `updatedTable` above? If the schema is unable to be inferred, we don't need to update the table.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104749098
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -124,11 +130,14 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
           fileFormatClass: Class[_ <: FileFormat],
           fileType: String): LogicalRelation = {
         val metastoreSchema = relation.tableMeta.schema
    -    val tableIdentifier =
    -      QualifiedTableName(relation.tableMeta.database, relation.tableMeta.identifier.table)
    +    val tableIdentifier = QualifiedTableName(
    +      relation.tableMeta.database, relation.tableMeta.identifier.table)
    +    val bucketSpec = None  // We don't support hive bucketed tables, only ones we write out.
    --- End diff --
    
    why keep this line?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73725 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73725/testReport)** for PR 16944 at commit [`1734598`](https://github.com/apache/spark/commit/1734598bc458fb9b6d6b3fae3640e1f576e602ae).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73720 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73720/testReport)** for PR 16944 at commit [`281bc6d`](https://github.com/apache/spark/commit/281bc6d53fbd0c0b5a99224d700b7d929397f090).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102554012
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,70 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +          val inferredSchema = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode != HiveCaseSensitiveInferenceMode.NEVER_INFER) {
    +            logInfo(s"Inferring case-sensitive schema for table $tableIdentifier")
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    +            if (fileType.equals("parquet")) {
    +              inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +            } else {
    +              inferred
    +            }
    +          } else {
    +            None
    +          }
    +
    +          // If configured, save the inferred case-sensitive schema to the table properties and
    +          // fetch the updated CatalogTable record for use in the LogicalRelation.
    +          val updatedCatalogTable = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode == HiveCaseSensitiveInferenceMode.INFER_AND_SAVE) {
    +            inferredSchema.flatMap { schema =>
    +              logInfo(s"Saving case-sensitive schema for table $tableIdentifier to table " +
    +                "properties")
    +              val updatedTable = catalogTable.copy(schema = schema)
    +              try {
    +                val catalog = sparkSession.sharedState.externalCatalog
    +                catalog.alterTable(updatedTable)
    +                Option(catalog.getTable(updatedTable.identifier.database.get,
    +                  updatedTable.identifier.table))
    +              } catch {
    +                case NonFatal(ex) =>
    +                  logError(s"Error saving case-sensitive schema for table $tableIdentifier: $ex")
    --- End diff --
    
    I copied this from the old schema inference mode. I'll update it to be a warning and include the inference mode.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73497 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73497/testReport)** for PR 16944 at commit [`b8d72b4`](https://github.com/apache/spark/commit/b8d72b467e01880c0870a553e27dd0ac771e5489).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Hi, @budde and @cloud-fan .
    Could you check the following Parquet Hive table issues?
    - https://issues.apache.org/jira/browse/SPARK-22306
    
    It seems to erase bucketing information from Metastore in `branch-2.2`.
    So far, Spark master branch looks okay.


---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102850475
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,45 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferredSchema = inferSchema(
    +            catalogTable,
    +            metastoreSchema,
    +            options,
    +            defaultSource,
    +            fileType,
    +            fileIndex)
    +
    +          // If configured, save the inferred case-sensitive schema to the table properties and
    +          // fetch the updated CatalogTable record for use in the LogicalRelation.
    +          val updatedCatalogTable = updateCatalogTable(catalogTable, inferredSchema)
    +
    +          val dataSchema = inferenceMode match {
    +            case (INFER_AND_SAVE | INFER_ONLY) if (!catalogTable.schemaPreservesCase) =>
    +              inferredSchema.getOrElse {
    +                logWarning(s"Unable to infer schema for table $tableIdentifier from file format " +
    +                  s"$defaultSource (inference mode: $inferenceMode); using metastore schema.")
    +                filteredMetastoreSchema
    +              }
    +            case _ =>
    +              filteredMetastoreSchema
    +          }
    --- End diff --
    
    I'll see if I can clean this up some more. It gets a bit messy since we'll want to replace ```catalogTable``` in the ```LogicalRelation``` returned by this method with the updated catalog table returned by ```updateTableCatalog()``` if the schema is successfully written to the metastore. Essentially, we need to return an ```Option[StructType]``` representing if the schema was inferred or not and an ```Option[CatalogTable]``` representing if the catalog table was updated.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73393 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73393/testReport)** for PR 16944 at commit [`9b0b2bb`](https://github.com/apache/spark/commit/9b0b2bb3fbc7db9e71b3342014b729568290dffd).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73808 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73808/testReport)** for PR 16944 at commit [`514ae06`](https://github.com/apache/spark/commit/514ae06e1dbe2640091c90d55354c3500857e6e2).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    LGTM


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104759937
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{
    +  Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      tableName: String,
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      hiveExternalCatalogTable(
    +        tableName,
    +        serde,
    +        dir.getAbsolutePath,
    +        lowerCaseSchema(schema),
    +        partitionCols.map(_.toLowerCase)),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      val partitions = (0 until NUM_RECORDS).map { idx =>
    +         hiveCatalogPartition(dir.getAbsolutePath, serde, partitionCols, idx)
    +      }.toSeq
    +      client.createPartitions(DATABASE, tableName, partitions, true)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, tableName).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, tableName)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (String, Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val tableName = "test_table"
    +        val schema = setupExternalTable(tableName, fileType, fields, partCols, dir)
    +        try f(
    +          tableName,
    +          fields,
    +          partCols,
    +          schema) finally spark.sql(s"DROP TABLE IF EXISTS $tableName")
    +      }
    +    }
    +  }
    +
    +  private def withFileTypes(f: (String) => Unit): Unit
    --- End diff --
    
    embed this into `withTestTables`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104252086
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -159,6 +165,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               Some(partitionSchema))
     
             val logicalRelation = cached.getOrElse {
    +          val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    --- End diff --
    
    Ah, I see. I missed that this filtering was removed in #17015. The existing ```metastoreSchema``` val can just be used then.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74083 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74083/testReport)** for PR 16944 at commit [`6542631`](https://github.com/apache/spark/commit/6542631bee813b517befe5ae548ae5abe27d747f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103051801
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -226,6 +258,41 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
       }
     
    +  private def inferSchema(
    +      metastoreSchema: StructType,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileType: String,
    +      fileIndex: FileIndex): Option[StructType] = {
    +    val inferred = fileFormat.inferSchema(
    +      sparkSession,
    +      options,
    +      fileIndex.listFiles(Nil).flatMap(_.files))
    +    if (fileType.equals("parquet")) {
    +      inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +    } else {
    +      inferred
    +    }
    +  }
    +
    +  private def updateCatalogTable(
    +      catalogTable: CatalogTable,
    +      inferredSchema: Option[StructType]): Option[CatalogTable] = try {
    +    inferredSchema.flatMap { schema =>
    --- End diff --
    
    We should be able to just make this a ```map``` though


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104530169
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +229,55 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    --- End diff --
    
    nit: `relation.tableMeta.identifier.unquotedString` is better, which also carries database name.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102553568
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -181,7 +186,8 @@ case class CatalogTable(
         viewText: Option[String] = None,
         comment: Option[String] = None,
         unsupportedFeatures: Seq[String] = Seq.empty,
    -    tracksPartitionsInCatalog: Boolean = false) {
    +    tracksPartitionsInCatalog: Boolean = false,
    +    schemaPreservesCase: Boolean = true) {
    --- End diff --
    
    I considered taking this approach but I think adding this as a parameter is more explicit and less flaky. I share your concern that adding more and more parameters to CatalogTable could make this less-usable, especially since params like ```schemaPreservesCase``` really only matter when dealing with Hive tables.
    
    However, I don't think dumping more and more parameters into ```properties``` is a great solution either. As you've pointed out, we would need to filter out the properties only used internally by Spark before writing them to the catalog. HiveExternalCatalog already filters out Spark SQL-specific properties from the CatalogTable returned by HiveClient. Adding additional internal properties would put us in a place where properties contains:
    
    - Actual properties key/value pairs returned from the Hive metastore table.
    - Spark SQL-specific properties that are stored in the Hive metastore table but filtered out by HiveExternalCatalog when used by Spark internally. These properties must be restored before writing back.
    - Spark SQL internal-only properties that are added after reading the table from the metastore and must be removed before writing it.
    
    Which isn't even to mention that we'll have to be serializing/deserializing this value to and from a (String, String) pair just to pass information between ```HiveExternalCatalog``` and ```HiveMetastoreCatalog```.
    
    I think that if CatalogTable ends up with too many datasource-specific internal parameters then maybe it makes more sense to introduce a new Map element, e.g. ```internalProperties```, so these don't get mixed in with the table properties.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104529452
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala ---
    @@ -104,6 +104,8 @@ abstract class ExternalCatalog {
        */
       def alterTable(tableDefinition: CatalogTable): Unit
     
    +  def alterTableSchema(db: String, table: String, schema: StructType): Unit
    --- End diff --
    
    please add some comments. One thing I wanna highlight is: the caller side is responsible for guarantee that the new schema still contains previous partition columns and bucket columns.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @dongjoon-hyun This behavior should be fixed by #17249. I'll amend this change to #17229 as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103068501
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    --- End diff --
    
    `dir =>` -> `table =>` or `_ =>` since it is not used below.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104534984
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +229,55 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    --- End diff --
    
    Will do


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74187/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103049381
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,25 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    +    val INFER_AND_SAVE, INFER_ONLY, NEVER_INFER = Value
    +  }
    +
    +  val HIVE_CASE_SENSITIVE_INFERENCE = buildConf("spark.sql.hive.caseSensitiveInferenceMode")
    +    .doc("Sets the action to take when a case-sensitive schema cannot be read from a Hive " +
    +      "table's properties. Although Spark SQL itself is not case-sensitive, Hive compatible file " +
    +      "formats such as Parquet are. Spark SQL must use a case-preserving schema when querying " +
    +      "any table backed by files containing case-sensitive field names or queries may not return " +
    +      "accurate results. Valid options include INFER_AND_SAVE (infer the case-sensitive " +
    --- End diff --
    
    Will fix


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73707 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73707/testReport)** for PR 16944 at commit [`a2658fc`](https://github.com/apache/spark/commit/a2658fced7886f82701739bc3a1ff5c7335a5bc8).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103116521
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    --- End diff --
    
    Since ```dir``` isn't being used in any of the tests I'm just going to roll it in to ```withTempDir()``` and change the type of the code block that function takes to ```() => Unit```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102371290
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -690,10 +696,10 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
               "different from the schema when this table was created by Spark SQL" +
               s"(${schemaFromTableProps.simpleString}). We have to fall back to the table schema " +
               "from Hive metastore which is not case preserving.")
    -        hiveTable
    +        hiveTable.copy(schemaPreservesCase = false)
    --- End diff --
    
    this is not only about case-preserving, maybe we should leave it unchanged


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104598894
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala ---
    @@ -476,46 +476,6 @@ object ParquetFileFormat extends Logging {
       }
     
       /**
    -   * Reconciles Hive Metastore case insensitivity issue and data type conflicts between Metastore
    -   * schema and Parquet schema.
    -   *
    -   * Hive doesn't retain case information, while Parquet is case sensitive. On the other hand, the
    -   * schema read from Parquet files may be incomplete (e.g. older versions of Parquet doesn't
    -   * distinguish binary and string).  This method generates a correct schema by merging Metastore
    -   * schema data types and Parquet schema field names.
    -   */
    -  def mergeMetastoreParquetSchema(
    -      metastoreSchema: StructType,
    -      parquetSchema: StructType): StructType = {
    -    def schemaConflictMessage: String =
    -      s"""Converting Hive Metastore Parquet, but detected conflicting schemas. Metastore schema:
    -         |${metastoreSchema.prettyJson}
    -         |
    -         |Parquet schema:
    -         |${parquetSchema.prettyJson}
    -       """.stripMargin
    -
    -    val mergedParquetSchema = mergeMissingNullableFields(metastoreSchema, parquetSchema)
    --- End diff --
    
    Oversight on my part. Removed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74225 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74225/testReport)** for PR 16944 at commit [`56fbc7e`](https://github.com/apache/spark/commit/56fbc7e0db2f3997016ff95dfd65afdf5cb891ec).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73123/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Hi, @budde and @cloud-fan .
    
    I met the following situation with Apache master after this commit. Could you check the following case? Previously, Apache Spark shows the correct result.
    
    ```scala
    sql("CREATE TABLE t1(a string, b string) PARTITIONED BY (day string, hour string) STORED AS PARQUET").show
    
    sql("INSERT INTO TABLE t1 PARTITION (day = '1', hour = '01' ) VALUES (100, 200)").show
    
    sql("SELECT a, b FROM t1").show
    +---+---+
    |  a|  b|
    +---+---+
    |100|200|
    +---+---+
    ```
    
    ```sql
    hive> ALTER TABLE t1 ADD COLUMNS (dummy string);
    ```
    
    ```scala
    sql("SELECT a, b FROM t1").show
    org.apache.hadoop.hive.ql.metadata.HiveException: Unable to alter table. partition keys can not be changed.
    ...
    +---+----+
    |  a|   b|
    +---+----+
    |100|null|
    +---+----+
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74051 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74051/testReport)** for PR 16944 at commit [`1e3af23`](https://github.com/apache/spark/commit/1e3af2355bce19c55e797b8d3ef139eba6da944a).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101560890
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,23 +161,49 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    +          val filteredMetastoreSchema = StructType(metastoreSchema
                   .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
     
    +          val inferenceMode = sparkSession.sessionState.conf.schemaInferenceMode
    +          val dataSchema = if (inferenceMode != "NEVER_INFER" &&
    +              !catalogTable.schemaFromTableProps) {
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    +            val merged = if (fileType.equals("parquet")) {
    +              inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +            } else {
    +              inferred
    --- End diff --
    
    I took this from how the schema was inferred in HiveMetastoreCatalog prior to 2.1.0. Only ParquetFileFormat has a merge method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74142 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74142/testReport)** for PR 16944 at commit [`fd78c65`](https://github.com/apache/spark/commit/fd78c65bff5fc02d96f4f116d21bf6e7efd65479).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Updated per feedback from @ericl:
    
    - Added comment with additional context to ```HIVE_CASE_SENSITIVE_INFERENCE``` in SQLConf.scala
    - Removed default value test for ```HIVE_CASE_SENSITIVE_INFERENCE``` from SQLConfSuite.scala (was primarily there for dev purposes)
    - Refactor changes to ```convertToLogicalRelation()``` as helper functions to simplify this function a little
    - Added the configured inference mode to several log lines
    
    I also reworked a lot of the comparisons involving the ```HiveCaseInsensitiveInferenceMode``` enumerations to be ```match { }``` blocks as I think this is more consistent with typical Scala conventions. I can change these back to ```if { }``` blocks though if consensus is that it's clearer.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104247964
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -195,17 +211,19 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               fileFormatClass,
               None)
             val logicalRelation = cached.getOrElse {
    +          val (dataSchema, updatedTable)
    +            = inferIfNeeded(relation, options, metastoreSchema, fileFormat)
               val created =
                 LogicalRelation(
                   DataSource(
                     sparkSession = sparkSession,
                     paths = rootPath.toString :: Nil,
    -                userSpecifiedSchema = Some(metastoreSchema),
    +                userSpecifiedSchema = Option(dataSchema),
                     // We don't support hive bucketed tables, only ones we write out.
                     bucketSpec = None,
                     options = options,
                     className = fileType).resolveRelation(),
    -              catalogTable = Some(relation.tableMeta))
    +               catalogTable = Option(updatedTable))
    --- End diff --
    
    Will fix


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74026 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74026/testReport)** for PR 16944 at commit [`79295ae`](https://github.com/apache/spark/commit/79295aedbe9f55932b5a23a5072103ec7152d045).
     * This patch **fails from timeout after a configured wait of \`250m\`**.
     * This patch **does not merge cleanly**.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r105286553
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.DATASOURCE_SCHEMA_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType = {
    +    StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      CatalogTable(
    +        identifier = TableIdentifier(table = TEST_TABLE_NAME, database = Option(DATABASE)),
    +        tableType = CatalogTableType.EXTERNAL,
    +        storage = CatalogStorageFormat(
    +          locationUri = Option(new java.net.URI(dir.getAbsolutePath)),
    +          inputFormat = serde.inputFormat,
    +          outputFormat = serde.outputFormat,
    +          serde = serde.serde,
    +          compressed = false,
    +          properties = Map("serialization.format" -> "1")),
    +        schema = schema,
    +        provider = Option("hive"),
    +        partitionColumnNames = partitionCols.map(_.toLowerCase),
    +        properties = Map.empty),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      spark.catalog.recoverPartitions(TEST_TABLE_NAME)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val schema = setupExternalTable(fileType, fields, partCols, dir)
    +        withTable(TEST_TABLE_NAME) { f(fields, partCols, schema) }
    +      }
    +    }
    +  }
    +
    +  private def withFileTypes(f: (String) => Unit): Unit
    +    = Seq(ORC_FILE_TYPE, PARQUET_FILE_TYPE).foreach(f)
    +
    +  private def withInferenceMode(mode: InferenceMode)(f: => Unit): Unit = {
    +    withSQLConf(
    +      HiveUtils.CONVERT_METASTORE_ORC.key -> "true",
    +      SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key -> mode.toString)(f)
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  private def testFieldQuery(fields: Seq[String]): Unit = {
    +    if (!fields.isEmpty) {
    +      val query = s"SELECT * FROM ${TEST_TABLE_NAME} WHERE ${Random.shuffle(fields).head} >= 0"
    +      assert(spark.sql(query).count == NUM_RECORDS)
    +    }
    +  }
    +
    +  private def testTableSchema(expectedSchema: StructType): Unit
    +    = assert(spark.table(TEST_TABLE_NAME).schema == expectedSchema)
    +
    +  withFileTypes { fileType =>
    +    test(s"$fileType: schema should be inferred and saved when INFER_AND_SAVE is specified") {
    +      withInferenceMode(INFER_AND_SAVE) {
    +        withTestTables(fileType) { (fields, partCols, schema) =>
    +          testFieldQuery(fields)
    +          testFieldQuery(partCols)
    +          testTableSchema(schema)
    +
    +          // Verify the catalog table now contains the updated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(catalogTable.schemaPreservesCase)
    +          assert(catalogTable.schema == schema)
    +          assert(catalogTable.partitionColumnNames == partCols.map(_.toLowerCase))
    --- End diff --
    
    When we infer the data schema, we have to list all the files first, which means we already know all the partition paths.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73863 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73863/testReport)** for PR 16944 at commit [`2b2dbe8`](https://github.com/apache/spark/commit/2b2dbe8fbeea1b0a999303acb9358c9584d73b1b).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103049838
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,25 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    --- End diff --
    
    Not in the current version of the code. I'll remove it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102550641
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala ---
    @@ -270,4 +271,9 @@ class SQLConfSuite extends QueryTest with SharedSQLContext {
         val e2 = intercept[AnalysisException](spark.conf.unset(SCHEMA_STRING_LENGTH_THRESHOLD.key))
         assert(e2.message.contains("Cannot modify the value of a static config"))
       }
    +
    +  test("Default value of HIVE_CASE_SENSITIVE_INFERENCE") {
    +    assert(spark.sessionState.conf.caseSensitiveInferenceMode ==
    +      HiveCaseSensitiveInferenceMode.INFER_AND_SAVE)
    --- End diff --
    
    IMO this test isn't really needed. You already have typo-safety via the enumeration check.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104084674
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -195,17 +207,20 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               fileFormatClass,
               None)
             val logicalRelation = cached.getOrElse {
    +          val fileIndex = new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    --- End diff --
    
    This `InMemoryFileIndex` will list leaf files even `inferenceMode` is `NEVER_INFER`. Isn't it unnecessary?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104249556
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -169,16 +179,22 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
                 }
               }
     
    +          val (dataSchema, updatedTable) = inferIfNeeded(
    +            relation,
    +            options,
    +            filteredMetastoreSchema,
    +            fileFormat,
    +            Option(fileIndex))
    +
               val fsRelation = HadoopFsRelation(
                 location = fileIndex,
                 partitionSchema = partitionSchema,
    -            dataSchema = relation.tableMeta.dataSchema,
    +            dataSchema = dataSchema,
                 // We don't support hive bucketed tables, only ones we write out.
                 bucketSpec = None,
    -            fileFormat = fileFormatClass.newInstance(),
    +            fileFormat = fileFormat,
                 options = options)(sparkSession = sparkSession)
    -
    -          val created = LogicalRelation(fsRelation, catalogTable = Some(relation.tableMeta))
    +          val created = LogicalRelation(fsRelation, catalogTable = Option(updatedTable))
    --- End diff --
    
    For clarification, ```Some(null)``` is preferred over ```None```?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104750141
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +229,55 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(relation.tableMeta.location)
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = fileFormat
    +        .inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        .map(mergeWithMetastoreSchema(relation.tableMeta.schema, _))
    --- End diff --
    
    makes sense


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r105270783
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,305 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.DATASOURCE_SCHEMA_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType = {
    +    StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      CatalogTable(
    +        identifier = TableIdentifier(table = TEST_TABLE_NAME, database = Option(DATABASE)),
    +        tableType = CatalogTableType.EXTERNAL,
    +        storage = CatalogStorageFormat(
    +          locationUri = Option(new java.net.URI(dir.getAbsolutePath)),
    +          inputFormat = serde.inputFormat,
    +          outputFormat = serde.outputFormat,
    +          serde = serde.serde,
    +          compressed = false,
    +          properties = Map("serialization.format" -> "1")),
    +        schema = schema,
    +        provider = Option("hive"),
    +        partitionColumnNames = partitionCols.map(_.toLowerCase),
    +        properties = Map.empty),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      spark.catalog.recoverPartitions(TEST_TABLE_NAME)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(DATASOURCE_SCHEMA_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val schema = setupExternalTable(fileType, fields, partCols, dir)
    +        withTable(TEST_TABLE_NAME) { f(fields, partCols, schema) }
    +      }
    +    }
    +  }
    +
    +  private def withFileTypes(f: (String) => Unit): Unit
    +    = Seq(ORC_FILE_TYPE, PARQUET_FILE_TYPE).foreach(f)
    +
    +  private def withInferenceMode(mode: InferenceMode)(f: => Unit): Unit = {
    +    withSQLConf(
    +      HiveUtils.CONVERT_METASTORE_ORC.key -> "true",
    +      SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key -> mode.toString)(f)
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  private def testFieldQuery(fields: Seq[String]): Unit = {
    +    if (!fields.isEmpty) {
    +      val query = s"SELECT * FROM ${TEST_TABLE_NAME} WHERE ${Random.shuffle(fields).head} >= 0"
    +      assert(spark.sql(query).count == NUM_RECORDS)
    +    }
    +  }
    +
    +  private def testTableSchema(expectedSchema: StructType): Unit
    +    = assert(spark.table(TEST_TABLE_NAME).schema == expectedSchema)
    +
    +  withFileTypes { fileType =>
    +    test(s"$fileType: schema should be inferred and saved when INFER_AND_SAVE is specified") {
    +      withInferenceMode(INFER_AND_SAVE) {
    +        withTestTables(fileType) { (fields, partCols, schema) =>
    +          testFieldQuery(fields)
    +          testFieldQuery(partCols)
    +          testTableSchema(schema)
    +
    +          // Verify the catalog table now contains the updated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(catalogTable.schemaPreservesCase)
    +          assert(catalogTable.schema == schema)
    +          assert(catalogTable.partitionColumnNames == partCols.map(_.toLowerCase))
    --- End diff --
    
    Would we need to inspect the partition paths to do this? Is there another way that I'm missing?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Updated to resolve dependency conflict


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    I think I'm a little unclear still on what exact components you are using-- you're using Spark SQL via ```spark-shell``` to create the table, then using Hive to alter it, then querying the table again via ```spark-shell```? Is this an external metastore or one managed by Spark locally?
    
    If the table is created with Spark, then Spark should be storing the schema under the table properties at that point. This was the behavior prior to this change as well. If the table schema is changed by another application that does not alter the schema in the table properties then Spark SQL the behavior *should* be the same as it was in 2.1.0. 
    
    The fact that you got the exception at all though seems to indicate that the schema wasn't properly read from the table properties or that some state wasn't managed properly. I'm going to try to recreate this locally to figure out what is happening here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73131/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103116519
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    --- End diff --
    
    I'll shorten this


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @cloud-fan Updated per your feedback. Mind taking a look at ```alterTableSchema()``` and checking that the change here is what you were looking for?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103050192
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -226,6 +258,41 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
       }
     
    +  private def inferSchema(
    +      metastoreSchema: StructType,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileType: String,
    +      fileIndex: FileIndex): Option[StructType] = {
    +    val inferred = fileFormat.inferSchema(
    +      sparkSession,
    +      options,
    +      fileIndex.listFiles(Nil).flatMap(_.files))
    +    if (fileType.equals("parquet")) {
    +      inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +    } else {
    +      inferred
    +    }
    +  }
    +
    +  private def updateCatalogTable(
    +      catalogTable: CatalogTable,
    +      inferredSchema: Option[StructType]): Option[CatalogTable] = try {
    +    inferredSchema.flatMap { schema =>
    --- End diff --
    
    We want the return value of this operation. It'll be the updated ```CatalogTable``` record if the update is successful.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103068277
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    --- End diff --
    
    Oh. nvm, you already use `withTempDir`.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Thanks, @ericl. Is there anybody else you'd suggest pinging to take a look at this and ultimately get it merged?
    
    Re-pinging @viirya to review latest updates addressing his previous feedback.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101461466
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,17 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  val HIVE_SCHEMA_INFERENCE_MODE = buildConf("spark.sql.hive.schemaInferenceMode")
    --- End diff --
    
    Is there always some schema stored for a table in the metastore? Consider including something about case-sensitivity in the conf name to distinguish it from general schema inference.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74225 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74225/testReport)** for PR 16944 at commit [`56fbc7e`](https://github.com/apache/spark/commit/56fbc7e0db2f3997016ff95dfd65afdf5cb891ec).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104795371
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalCatalogSuite.scala ---
    @@ -71,4 +72,19 @@ class HiveExternalCatalogSuite extends ExternalCatalogSuite {
         assert(!rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_PROVIDER))
         assert(DDLUtils.isHiveTable(externalCatalog.getTable("db1", "hive_tbl")))
       }
    +
    +  test("alter table schema should save case-sensitive schema to properties") {
    --- End diff --
    
    as long as the returned table schema is corrected, we don't need to care about these special table properties. We already have a lot of tests to guarantee that table schema always matches the special table properties.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101456277
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -186,8 +212,7 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
                 fileFormat = defaultSource,
                 options = options)(sparkSession = sparkSession)
     
    -          val created = LogicalRelation(relation,
    -            catalogTable = Some(metastoreRelation.catalogTable))
    +          val created = LogicalRelation(relation, catalogTable = Some(catalogTable))
    --- End diff --
    
    Once catalog table info is altered, shall we use the updated catalog table?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103068259
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    --- End diff --
    
    As it is external table, shall we delete the dir along with dropping the table?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Retest this please


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104081966
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -195,17 +207,20 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               fileFormatClass,
               None)
             val logicalRelation = cached.getOrElse {
    +          val fileIndex = new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +          val (dataSchema, updatedTable)
    +            = inferIfNeeded(relation, options, relation.tableMeta.schema, fileFormat, fileIndex)
    --- End diff --
    
    nit: `relation.tableMeta.schema` -> `metastoreSchema`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74142 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74142/testReport)** for PR 16944 at commit [`fd78c65`](https://github.com/apache/spark/commit/fd78c65bff5fc02d96f4f116d21bf6e7efd65479).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73393/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74026 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74026/testReport)** for PR 16944 at commit [`79295ae`](https://github.com/apache/spark/commit/79295aedbe9f55932b5a23a5072103ec7152d045).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104531323
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +229,55 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(relation.tableMeta.location)
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = fileFormat
    +        .inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        .map(mergeWithMetastoreSchema(relation.tableMeta.schema, _))
    --- End diff --
    
    can we just do `fileFormat.inferSchema(...).map(dataSchema => StructType(dataSchema ++ relation.tableMeta.partitionSchema))`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104244767
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +235,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    --- End diff --
    
    is it really needed?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74029/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104244137
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -159,6 +165,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               Some(partitionSchema))
     
             val logicalRelation = cached.getOrElse {
    +          val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    --- End diff --
    
    how about `val metastoreDataSchema = relation.tableMeta.dataSchema`?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @viirya I've updated the PR to include the initial catalog table checks you've suggested in the [```setupCaseSensitiveTable()```](https://github.com/apache/spark/pull/16944/files#diff-f3b945ffe3f3f57b520b655c8b918a80R124) helper method.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73014 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73014/testReport)** for PR 16944 at commit [`ed6ea2e`](https://github.com/apache/spark/commit/ed6ea2ebbaff7f29b7b13f4e2fe8ce4affe9ce38).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @cloud-fan Thanks for taking the time to review this! I'll get a PR open to backport it.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102554381
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,70 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +          val inferredSchema = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode != HiveCaseSensitiveInferenceMode.NEVER_INFER) {
    +            logInfo(s"Inferring case-sensitive schema for table $tableIdentifier")
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    +            if (fileType.equals("parquet")) {
    +              inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +            } else {
    +              inferred
    +            }
    +          } else {
    +            None
    +          }
    +
    +          // If configured, save the inferred case-sensitive schema to the table properties and
    +          // fetch the updated CatalogTable record for use in the LogicalRelation.
    +          val updatedCatalogTable = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode == HiveCaseSensitiveInferenceMode.INFER_AND_SAVE) {
    +            inferredSchema.flatMap { schema =>
    +              logInfo(s"Saving case-sensitive schema for table $tableIdentifier to table " +
    +                "properties")
    +              val updatedTable = catalogTable.copy(schema = schema)
    +              try {
    +                val catalog = sparkSession.sharedState.externalCatalog
    +                catalog.alterTable(updatedTable)
    +                Option(catalog.getTable(updatedTable.identifier.database.get,
    +                  updatedTable.identifier.table))
    +              } catch {
    +                case NonFatal(ex) =>
    +                  logError(s"Error saving case-sensitive schema for table $tableIdentifier: $ex")
    +                  None
    +              }
    +            }
    +          } else {
    +            None
    +          }
    +
    +          val dataSchema = if (!catalogTable.schemaPreservesCase) {
    +            inferredSchema.getOrElse {
    +              logWarning(s"Unable to infer schema for table $tableIdentifier from file format " +
    +                s"$defaultSource; using metastore schema.")
    +              filteredMetastoreSchema
    +            }
    +          } else {
    +            filteredMetastoreSchema
    +          }
    --- End diff --
    
    I started by trying to keep this as consistent as possible with the previous schema inference code but I agree that this function is a bit unwieldy. I'll refactor some of this code into helper functions.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104263454
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    --- End diff --
    
    `fallbackSchema` is always `relation.tableMeta.schema`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73497/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102843314
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,45 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferredSchema = inferSchema(
    +            catalogTable,
    +            metastoreSchema,
    +            options,
    +            defaultSource,
    +            fileType,
    +            fileIndex)
    +
    +          // If configured, save the inferred case-sensitive schema to the table properties and
    +          // fetch the updated CatalogTable record for use in the LogicalRelation.
    +          val updatedCatalogTable = updateCatalogTable(catalogTable, inferredSchema)
    +
    +          val dataSchema = inferenceMode match {
    +            case (INFER_AND_SAVE | INFER_ONLY) if (!catalogTable.schemaPreservesCase) =>
    +              inferredSchema.getOrElse {
    +                logWarning(s"Unable to infer schema for table $tableIdentifier from file format " +
    +                  s"$defaultSource (inference mode: $inferenceMode); using metastore schema.")
    +                filteredMetastoreSchema
    +              }
    +            case _ =>
    +              filteredMetastoreSchema
    +          }
    --- End diff --
    
    I'm still finding this a bit awkward to read because the inferenceMode config flag is referenced in three separate places. How about centralizing the logic a little, moving the config checks to the body of this function?
    
    ```
    val shouldInferSchema = !catalogTable.schemaPreservesCase && inferenceMode != NEVER_INFER
    val dataSchema = if (shouldInferSchema) {
      val inferredSchema = inferSchema(...)
      if (inferenceMode == INFER_AND_SAVE) {
        try {
           updateCatalogTableSchema(...)
        } catch {
           ...
        }
      }
      inferredSchema
    } else {
      filteredMetastoreSchema
    }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73444 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73444/testReport)** for PR 16944 at commit [`e1ca7c8`](https://github.com/apache/spark/commit/e1ca7c823a2262de3e2e3da451e43437facc064f).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74029 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74029/testReport)** for PR 16944 at commit [`66ed1fc`](https://github.com/apache/spark/commit/66ed1fcaa76544c356a65fabed886271ef1b1313).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Few small comments left. LGTM


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101461535
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,23 +161,49 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    +          val filteredMetastoreSchema = StructType(metastoreSchema
                   .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
     
    +          val inferenceMode = sparkSession.sessionState.conf.schemaInferenceMode
    +          val dataSchema = if (inferenceMode != "NEVER_INFER" &&
    +              !catalogTable.schemaFromTableProps) {
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    --- End diff --
    
    I'll add an info log here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104760055
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{
    +  Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      tableName: String,
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      hiveExternalCatalogTable(
    +        tableName,
    +        serde,
    +        dir.getAbsolutePath,
    +        lowerCaseSchema(schema),
    +        partitionCols.map(_.toLowerCase)),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      val partitions = (0 until NUM_RECORDS).map { idx =>
    +         hiveCatalogPartition(dir.getAbsolutePath, serde, partitionCols, idx)
    +      }.toSeq
    +      client.createPartitions(DATABASE, tableName, partitions, true)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, tableName).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, tableName)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (String, Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val tableName = "test_table"
    +        val schema = setupExternalTable(tableName, fileType, fields, partCols, dir)
    +        try f(
    --- End diff --
    
    Will do


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73444 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73444/testReport)** for PR 16944 at commit [`e1ca7c8`](https://github.com/apache/spark/commit/e1ca7c823a2262de3e2e3da451e43437facc064f).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73720 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73720/testReport)** for PR 16944 at commit [`281bc6d`](https://github.com/apache/spark/commit/281bc6d53fbd0c0b5a99224d700b7d929397f090).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by viirya <gi...@git.apache.org>.
Github user viirya commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101888278
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          // Test that the case-sensitive schema was storied as a table property after inference
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == caseSensitiveSchema)
    +
    +          // Verify the catalog table now contains the udpated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(catalogTable.schemaPreservesCase == true)
    +          assert(catalogTable.schema == caseSensitiveSchema)
    +          assert(catalogTable.partitionColumnNames == Seq("partcol1", "partcol2"))
    +          val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_SCHEMA_NUMPARTS))
    +        }
    +      }
    +    }
    +  }
    +
    +  test("Schema should be inferred but not stored when INFER_ONLY is specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_ONLY.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          val existingSchema = spark.sql(SELECT_ALL_QUERY).schema
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == existingSchema)
    +          // Catalog table shouldn't be altered
    +          assert(externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) == originalTable)
    +        }
    +      }
    +    }
    +  }
    +
    +  test("Schema should not be inferred when NEVER_INFER is specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.NEVER_INFER.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          // Only check the schema returned by SELECT * as other queries will break
    --- End diff --
    
    Can we also add queries that break? We can intercept their exceptions.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74133 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74133/testReport)** for PR 16944 at commit [`aa433ec`](https://github.com/apache/spark/commit/aa433ec794c156c513897001427ccf260a1c3479).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104270925
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    +          case _ =>
    +            schema
    +        }
    +      }
    +
    +      inferredSchema match {
    +        case Some(schema) =>
    +          if (inferenceMode == INFER_AND_SAVE) {
    +            updateCatalogSchema(relation.tableMeta.identifier, schema)
    --- End diff --
    
    ah the `inferredSchema` contains partition schema for parquet because we call `ParquetFileFormat.mergeMetastoreParquetSchema`, but this doesn't work for other formats(orc) and make the code hard to understand. We should do this more explicitly.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73863/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102369793
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -181,7 +186,8 @@ case class CatalogTable(
         viewText: Option[String] = None,
         comment: Option[String] = None,
         unsupportedFeatures: Seq[String] = Seq.empty,
    -    tracksPartitionsInCatalog: Boolean = false) {
    +    tracksPartitionsInCatalog: Boolean = false,
    +    schemaPreservesCase: Boolean = true) {
    --- End diff --
    
    shall we create a special table property? `object CatalogTable` defines some specify properties for view and we can follow it. If we keeps adding more parameters, we may blow up the `CatalogTable` one day...


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74187 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74187/testReport)** for PR 16944 at commit [`8e1e2aa`](https://github.com/apache/spark/commit/8e1e2aa366f9b77242656331279410e637e727cd).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103116512
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,51 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +          val shouldInfer = (inferenceMode != NEVER_INFER) && !catalogTable.schemaPreservesCase
    +
    +          val (dataSchema, updatedTable) = if (shouldInfer) {
    +            logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +              s"$inferenceMode)")
    +            val inferredSchema
    +              = inferSchema(metastoreSchema, options, defaultSource, fileType, fileIndex)
    +
    +            // If configured, save the inferred case-sensitive schema to the table properties and
    +            // fetch the updated CatalogTable record for use in the LogicalRelation.
    +            val updatedTable = if (inferenceMode == INFER_AND_SAVE) {
    +              updateCatalogTable(catalogTable, inferredSchema).getOrElse {
    +                logWarning(s"Unable to save case-sensitive schema for table $tableName " +
    +                  s"(inference mode: $inferenceMode)")
    +                catalogTable
    +              }
    +            } else {
    +              catalogTable
    +            }
    +            val schema = inferredSchema.getOrElse {
    --- End diff --
    
    See my latest comment at line 281. I'll rework this section.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73393 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73393/testReport)** for PR 16944 at commit [`9b0b2bb`](https://github.com/apache/spark/commit/9b0b2bb3fbc7db9e71b3342014b729568290dffd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73014 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73014/testReport)** for PR 16944 at commit [`ed6ea2e`](https://github.com/apache/spark/commit/ed6ea2ebbaff7f29b7b13f4e2fe8ce4affe9ce38).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101605728
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,17 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  val HIVE_SCHEMA_INFERENCE_MODE = buildConf("spark.sql.hive.schemaInferenceMode")
    --- End diff --
    
    Will do


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102852766
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,45 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferredSchema = inferSchema(
    +            catalogTable,
    +            metastoreSchema,
    +            options,
    +            defaultSource,
    +            fileType,
    +            fileIndex)
    +
    +          // If configured, save the inferred case-sensitive schema to the table properties and
    +          // fetch the updated CatalogTable record for use in the LogicalRelation.
    +          val updatedCatalogTable = updateCatalogTable(catalogTable, inferredSchema)
    +
    +          val dataSchema = inferenceMode match {
    +            case (INFER_AND_SAVE | INFER_ONLY) if (!catalogTable.schemaPreservesCase) =>
    +              inferredSchema.getOrElse {
    +                logWarning(s"Unable to infer schema for table $tableIdentifier from file format " +
    +                  s"$defaultSource (inference mode: $inferenceMode); using metastore schema.")
    +                filteredMetastoreSchema
    +              }
    +            case _ =>
    +              filteredMetastoreSchema
    +          }
    --- End diff --
    
    You could return a tuple in that case. I think this would work, though now it is getting a bit messy too:
    
    ```
    val shouldInferSchema = !catalogTable.schemaPreservesCase && inferenceMode != NEVER_INFER
    val (dataSchema, updatedTable) = if (shouldInferSchema) {
      val inferredSchema = inferSchema(...)
      if (inferredSchema.isDefined && inferenceMode == INFER_AND_SAVE) {
        try {
           val newTable = updateCatalogTableSchema(...)
           (inferredSchema, newTable)
        } catch {
           ...
           (inferredSchema, catalogTable)
        }
      } else {
        (filteredMetastoreSchema, catalogTable)
      }
    } else {
      (filteredMetastoreSchema, catalogTable)
    }
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104263617
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    +          case _ =>
    +            schema
    +        }
    +      }
    +
    +      inferredSchema match {
    +        case Some(schema) =>
    +          if (inferenceMode == INFER_AND_SAVE) {
    +            updateCatalogSchema(relation.tableMeta.identifier, schema)
    --- End diff --
    
    so we actually want to update only the data schema of this table right?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73858 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73858/testReport)** for PR 16944 at commit [`d0652f9`](https://github.com/apache/spark/commit/d0652f93dd41ed91e753dabc755788f9a9b7f2c6).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #72959 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72959/testReport)** for PR 16944 at commit [`0bca163`](https://github.com/apache/spark/commit/0bca163a2ad53a93e629874a3cc0275532b25bac).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101908105
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,192 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +  }
    +
    +  // Create a test table used for a single unit test, with data stored in the specified directory.
    +  private def withTestTable(dir: File)(f: File => Unit): Unit = {
    +    setupCaseSensitiveTable(TEST_TABLE_NAME, dir)
    +    try f(dir) finally spark.sql(s"DROP TABLE IF EXISTS $TEST_TABLE_NAME")
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  test("Schema should be inferred and written to table properties when INFER_AND_SAVE is " +
    +    "specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_AND_SAVE.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          // Test that the case-sensitive schema was storied as a table property after inference
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == caseSensitiveSchema)
    +
    +          // Verify the catalog table now contains the udpated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(catalogTable.schemaPreservesCase == true)
    +          assert(catalogTable.schema == caseSensitiveSchema)
    +          assert(catalogTable.partitionColumnNames == Seq("partcol1", "partcol2"))
    +          val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    +          assert(rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_SCHEMA_NUMPARTS))
    +        }
    +      }
    +    }
    +  }
    +
    +  test("Schema should be inferred but not stored when INFER_ONLY is specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.INFER_ONLY.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          val existingSchema = spark.sql(SELECT_ALL_QUERY).schema
    +          assert(spark.sql(FIELD_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(PARTITION_COLUMN_QUERY).count == NUM_RECORDS)
    +          assert(spark.sql(SELECT_ALL_QUERY).schema == existingSchema)
    +          // Catalog table shouldn't be altered
    +          assert(externalCatalog.getTable(DATABASE, TEST_TABLE_NAME) == originalTable)
    +        }
    +      }
    +    }
    +  }
    +
    +  test("Schema should not be inferred when NEVER_INFER is specified") {
    +    withSQLConf(inferenceKey -> HiveCaseSensitiveInferenceMode.NEVER_INFER.toString) {
    +      withTempDir { dir =>
    +        withTestTable(dir) { dir =>
    +          val originalTable = externalCatalog.getTable(DATABASE, TEST_TABLE_NAME)
    +          // Only check the schema returned by SELECT * as other queries will break
    --- End diff --
    
    As mentioned in #16797 this issue actually won't cause exceptions, at least for Parquet data. The queries will simply return 0 results due to ParquetReadSupport using case-sensitive field resolution. If enabled, any pushed-down filter containing a case-sensitive field will also return 0 results since the lowercase filter field name won't match the case-sensitive Parquet column name.
    
    I'll put some thought towards whether this test can be made more robust in other ways.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @ericl: fixed the param doc string and tried to clean up ```createLogicalRelation()``` as you suggested.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73877 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73877/testReport)** for PR 16944 at commit [`64544ec`](https://github.com/apache/spark/commit/64544ecb96a92af605fdab12bedbf351998f65b0).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73361 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73361/testReport)** for PR 16944 at commit [`b78c8e1`](https://github.com/apache/spark/commit/b78c8e15b2f5aa2a8729b391ba631755c8419123).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104759088
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,352 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import scala.util.Random
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.{HiveSerDe, SQLConf}
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode.{
    +  Value => InferenceMode, _}
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    spark.sessionState.catalog.tableRelationCache.invalidateAll()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val client = externalCatalog.client
    +
    +  // Return a copy of the given schema with all field names converted to lower case.
    +  private def lowerCaseSchema(schema: StructType): StructType
    +    = StructType(schema.map(f => f.copy(name = f.name.toLowerCase)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      serde: HiveSerDe,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(new java.net.URI(location)),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Create a CatalogTablePartition instance for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(
    +      location: String,
    +      serde: HiveSerDe,
    +      partitionCols: Seq[String],
    +      index: Int): CatalogTablePartition = {
    +    val locationUri =
    +      new java.net.URI(location + "/" +  partitionCols.map(c => s"$c=$index").mkString("/") + "/")
    +    CatalogTablePartition(
    +      spec = partitionCols.map(_.toLowerCase -> index.toString).toMap,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(locationUri),
    +        inputFormat = serde.inputFormat,
    +        outputFormat = serde.outputFormat,
    +        serde = serde.serde,
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +  }
    +
    +  // Create a Hive external test table containing the given field and partition column names.
    +  // Returns a case-sensitive schema for the table.
    +  private def setupExternalTable(
    +      tableName: String,
    +      fileType: String,
    +      fields: Seq[String],
    +      partitionCols: Seq[String],
    +      dir: File): StructType = {
    +    // Treat all table fields as bigints...
    +    val structFields = fields.map { field =>
    +      StructField(
    +        name = field,
    +        dataType = LongType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "bigint").build())
    +    }
    +    // and all partition columns as ints
    +    val partitionStructFields = partitionCols.map { field =>
    +      StructField(
    +        // Partition column case isn't preserved
    +        name = field.toLowerCase,
    +        dataType = IntegerType,
    +        nullable = true,
    +        metadata = new MetadataBuilder().putString(HIVE_TYPE_STRING, "int").build())
    +    }
    +    val schema = StructType(structFields ++ partitionStructFields)
    +
    +    // Write some test data (partitioned if specified)
    +    val writer = spark.range(NUM_RECORDS)
    +      .selectExpr((fields ++ partitionCols).map("id as " + _): _*)
    +      .write
    +      .partitionBy(partitionCols: _*)
    +      .mode("overwrite")
    +    fileType match {
    +      case ORC_FILE_TYPE =>
    +       writer.orc(dir.getAbsolutePath)
    +      case PARQUET_FILE_TYPE =>
    +       writer.parquet(dir.getAbsolutePath)
    +    }
    +
    +    // Create Hive external table with lowercased schema
    +    val serde = HiveSerDe.serdeMap(fileType)
    +    client.createTable(
    +      hiveExternalCatalogTable(
    +        tableName,
    +        serde,
    +        dir.getAbsolutePath,
    +        lowerCaseSchema(schema),
    +        partitionCols.map(_.toLowerCase)),
    +      true)
    +
    +    // Add partition records (if specified)
    +    if (!partitionCols.isEmpty) {
    +      val partitions = (0 until NUM_RECORDS).map { idx =>
    +         hiveCatalogPartition(dir.getAbsolutePath, serde, partitionCols, idx)
    +      }.toSeq
    +      client.createPartitions(DATABASE, tableName, partitions, true)
    +    }
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have any Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, tableName).schemaPreservesCase)
    +    val rawTable = client.getTable(DATABASE, tableName)
    +    assert(rawTable.properties.filterKeys(_.startsWith(SPARK_SQL_PREFIX)) == Map.empty)
    +    schema
    +  }
    +
    +  private def withTestTables(
    +    fileType: String)(f: (String, Seq[String], Seq[String], StructType) => Unit): Unit = {
    +    // Test both a partitioned and unpartitioned Hive table
    +    val tableFields = Seq(
    +      (Seq("fieldOne"), Seq("partCol1", "partCol2")),
    +      (Seq("fieldOne", "fieldTwo"), Seq.empty[String]))
    +
    +    tableFields.foreach { case (fields, partCols) =>
    +      withTempDir { dir =>
    +        val tableName = "test_table"
    +        val schema = setupExternalTable(tableName, fileType, fields, partCols, dir)
    +        try f(
    +          tableName,
    +          fields,
    +          partCols,
    +          schema) finally spark.sql(s"DROP TABLE IF EXISTS $tableName")
    +      }
    +    }
    +  }
    +
    +  private def withFileTypes(f: (String) => Unit): Unit
    +    = Seq(ORC_FILE_TYPE, PARQUET_FILE_TYPE).foreach(f)
    +
    +  private def withInferenceMode(mode: InferenceMode)(f: => Unit): Unit = {
    +    withSQLConf(
    +      HiveUtils.CONVERT_METASTORE_ORC.key -> "true",
    +      SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key -> mode.toString)(f)
    +  }
    +
    +  private val inferenceKey = SQLConf.HIVE_CASE_SENSITIVE_INFERENCE.key
    +
    +  private def testFieldQuery(tableName: String, fields: Seq[String]): Unit = {
    +    if (!fields.isEmpty) {
    +      val query = s"SELECT * FROM $tableName WHERE ${Random.shuffle(fields).head} >= 0"
    +      assert(spark.sql(query).count == NUM_RECORDS)
    +    }
    +  }
    +
    +  private def testSchemaQuery(tableName: String, expectedSchema: StructType): Unit
    +    = assert(spark.sql(s"SELECT * FROM $tableName").schema == expectedSchema)
    +
    +  withFileTypes { fileType =>
    +    test(s"$fileType: schema should be inferred and saved when INFER_AND_SAVE is specified") {
    +      withInferenceMode(INFER_AND_SAVE) {
    +        withTestTables(fileType) { (tableName, fields, partCols, schema) =>
    +          testFieldQuery(tableName, fields)
    +          testFieldQuery(tableName, partCols)
    +          // Test that the case-sensitive schema was storied as a table property after inference
    +          testSchemaQuery(tableName, schema)
    +
    +          // Verify the catalog table now contains the updated schema and properties
    +          val catalogTable = externalCatalog.getTable(DATABASE, tableName)
    +          assert(catalogTable.schemaPreservesCase)
    +          assert(catalogTable.schema == schema)
    +          assert(catalogTable.partitionColumnNames == partCols.map(_.toLowerCase))
    +          val rawTable = client.getTable(DATABASE, tableName)
    +          assert(rawTable.properties.contains(HiveExternalCatalog.DATASOURCE_SCHEMA_NUMPARTS))
    --- End diff --
    
    Fair enough. ```HiveExternalCatalogSuite``` now tests that these properties are set and that's a more appropriate place to check this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73707/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101605711
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala ---
    @@ -163,6 +163,10 @@ case class BucketSpec(
      * @param tracksPartitionsInCatalog whether this table's partition metadata is stored in the
      *                                  catalog. If false, it is inferred automatically based on file
      *                                  structure.
    + * @param schemaFromTableProps Whether the schema field was obtained by parsing a case-sensitive
    --- End diff --
    
    Will do


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73131 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73131/testReport)** for PR 16944 at commit [`15c25e0`](https://github.com/apache/spark/commit/15c25e0fc23ec48ecd0fad7ca7c60d3a82d32a73).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Yes. It's Mac environment with `spark-shell` and `hive 1.2.1` fully locally. You can try that in your mac.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by dongjoon-hyun <gi...@git.apache.org>.
Github user dongjoon-hyun commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    I found that `NEVER_INFER` work.
    ```
    scala> sql("set spark.sql.hive.caseSensitiveInferenceMode=NEVER_INFER").show
    +--------------------+-----------+
    |                 key|      value|
    +--------------------+-----------+
    |spark.sql.hive.ca...|NEVER_INFER|
    +--------------------+-----------+
    
    scala> sql("SELECT a, b FROM t4").show
    +---+---+
    |  a|  b|
    +---+---+
    |100|200|
    +---+---+
    ```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/spark/pull/16944


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104250011
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -169,16 +179,22 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
                 }
               }
     
    +          val (dataSchema, updatedTable) = inferIfNeeded(
    +            relation,
    +            options,
    +            filteredMetastoreSchema,
    +            fileFormat,
    +            Option(fileIndex))
    +
               val fsRelation = HadoopFsRelation(
                 location = fileIndex,
                 partitionSchema = partitionSchema,
    -            dataSchema = relation.tableMeta.dataSchema,
    +            dataSchema = dataSchema,
                 // We don't support hive bucketed tables, only ones we write out.
                 bucketSpec = None,
    -            fileFormat = fileFormatClass.newInstance(),
    +            fileFormat = fileFormat,
                 options = options)(sparkSession = sparkSession)
    -
    -          val created = LogicalRelation(fsRelation, catalogTable = Some(relation.tableMeta))
    +          val created = LogicalRelation(fsRelation, catalogTable = Option(updatedTable))
    --- End diff --
    
    no, `Some(a)` is preferred over `Option(a)`, when we know `a` should not be null


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103051158
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -226,6 +258,41 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
       }
     
    +  private def inferSchema(
    +      metastoreSchema: StructType,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileType: String,
    +      fileIndex: FileIndex): Option[StructType] = {
    +    val inferred = fileFormat.inferSchema(
    +      sparkSession,
    +      options,
    +      fileIndex.listFiles(Nil).flatMap(_.files))
    +    if (fileType.equals("parquet")) {
    +      inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +    } else {
    +      inferred
    +    }
    +  }
    +
    +  private def updateCatalogTable(
    +      catalogTable: CatalogTable,
    +      inferredSchema: Option[StructType]): Option[CatalogTable] = try {
    +    inferredSchema.flatMap { schema =>
    +      logInfo(s"Saving case-sensitive schema for table ${catalogTable.identifier.table}")
    +      val updatedTable = catalogTable.copy(schema = schema)
    +      val catalog = sparkSession.sharedState.externalCatalog
    +      catalog.alterTable(updatedTable)
    +      Option(catalog.getTable(
    --- End diff --
    
    I think that should be fine. I had some concerns around the way ```HiveExternalCatalog``` mutates the raw ```CatalogTable``` returned by the metastore that I think pushed me towards fetching the table again but I really don't think that should matter since the original ```catalogTable``` was retrieved from ```HiveExternalCatalog``` as well.
    
    I'll just used ```updatedTable``` here.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103055530
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala ---
    @@ -510,8 +510,13 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat
         requireTableExists(db, tableDefinition.identifier.table)
         verifyTableProperties(tableDefinition)
     
    +    // Add table metadata such as table schema, partition columns, etc. if they aren't already
    +    // present.
    +    val withMetaProps = tableDefinition.copy(
    --- End diff --
    
    yea, we need to add a new interface in `ExternalCatalog`


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103116509
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -226,6 +258,41 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(metastoreRelation.output))
       }
     
    +  private def inferSchema(
    +      metastoreSchema: StructType,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileType: String,
    +      fileIndex: FileIndex): Option[StructType] = {
    +    val inferred = fileFormat.inferSchema(
    +      sparkSession,
    +      options,
    +      fileIndex.listFiles(Nil).flatMap(_.files))
    +    if (fileType.equals("parquet")) {
    +      inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +    } else {
    +      inferred
    +    }
    +  }
    +
    +  private def updateCatalogTable(
    +      catalogTable: CatalogTable,
    +      inferredSchema: Option[StructType]): Option[CatalogTable] = try {
    +    inferredSchema.flatMap { schema =>
    --- End diff --
    
    Actually, thinking about it, I think we'll want to update ```catalogTable.schema ``` any time we successfully infer a schema. This means this function can be converted to return ```Unit``` and a lot of the changes to ```convertToLogicalRelation``` can be simplified


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104251665
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -169,16 +179,22 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
                 }
               }
     
    +          val (dataSchema, updatedTable) = inferIfNeeded(
    +            relation,
    +            options,
    +            filteredMetastoreSchema,
    +            fileFormat,
    +            Option(fileIndex))
    +
               val fsRelation = HadoopFsRelation(
                 location = fileIndex,
                 partitionSchema = partitionSchema,
    -            dataSchema = relation.tableMeta.dataSchema,
    +            dataSchema = dataSchema,
                 // We don't support hive bucketed tables, only ones we write out.
                 bucketSpec = None,
    -            fileFormat = fileFormatClass.newInstance(),
    +            fileFormat = fileFormat,
                 options = options)(sparkSession = sparkSession)
    -
    -          val created = LogicalRelation(fsRelation, catalogTable = Some(relation.tableMeta))
    +          val created = LogicalRelation(fsRelation, catalogTable = Option(updatedTable))
    --- End diff --
    
    I don't think I was very clear. Hypothetically, if ```a``` were null when it shouldn't be:
    
    1. ```Some(a)``` will result in ```Some(null)```
    2. ```Option(a)``` will result in ```None```
    
    Based on your response, (1) is the preferred approach, meaning a runtime error will occur when attempting to use the (expected non-null) value rather than just thinking an optional value wasn't provided.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102554155
  
    --- Diff: sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
    @@ -296,6 +296,21 @@ object SQLConf {
           .longConf
           .createWithDefault(250 * 1024 * 1024)
     
    +  object HiveCaseSensitiveInferenceMode extends Enumeration {
    +    type HiveCaseSensitiveInferenceMode = Value
    +    val INFER_AND_SAVE, INFER_ONLY, NEVER_INFER = Value
    +  }
    +  val HIVE_CASE_SENSITIVE_INFERENCE = buildConf("spark.sql.hive.caseSensitiveInferenceMode")
    +    .doc("Configures the action to take when a case-sensitive schema cannot be read from a Hive " +
    +      "table's properties. Valid options include INFER_AND_SAVE (infer the case-sensitive " +
    --- End diff --
    
    Would you prefer this as a code comment or embedded in the doc string itself?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74196 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74196/testReport)** for PR 16944 at commit [`8e1e2aa`](https://github.com/apache/spark/commit/8e1e2aa366f9b77242656331279410e637e727cd).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74154 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74154/testReport)** for PR 16944 at commit [`837ba54`](https://github.com/apache/spark/commit/837ba544735fc344253699ef675849c1c603e835).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74050/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72959/
    Test FAILed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73877 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73877/testReport)** for PR 16944 at commit [`64544ec`](https://github.com/apache/spark/commit/64544ecb96a92af605fdab12bedbf351998f65b0).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104535285
  
    --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalog.scala ---
    @@ -104,6 +104,8 @@ abstract class ExternalCatalog {
        */
       def alterTable(tableDefinition: CatalogTable): Unit
     
    +  def alterTableSchema(db: String, table: String, schema: StructType): Unit
    --- End diff --
    
    Wasn't sure if I needed to add a doc comment here since this appears to be missing for most methods in this interface but I'll add one here since there are preconditions the caller must be aware of


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104270713
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    +          case _ =>
    +            schema
    +        }
    +      }
    +
    +      inferredSchema match {
    +        case Some(schema) =>
    +          if (inferenceMode == INFER_AND_SAVE) {
    +            updateCatalogSchema(relation.tableMeta.identifier, schema)
    --- End diff --
    
    the `inferredSchema` is data schema(excluding partition schema), and `updateCatalogSchema` will just update the table to this schema, which means we will lose partition schema after this.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74155 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74155/testReport)** for PR 16944 at commit [`8e1e2aa`](https://github.com/apache/spark/commit/8e1e2aa366f9b77242656331279410e637e727cd).
     * This patch **fails PySpark unit tests**.
     * This patch merges cleanly.
     * This patch adds no public classes.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104268313
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -169,16 +175,18 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
                 }
               }
     
    +          val (dataSchema, updatedTable) =
    +            inferIfNeeded(relation, options, metastoreSchema, fileFormat, Option(fileIndex))
    --- End diff --
    
    ```metastoreSchema``` is set to ```relation.tableMeta.schema``` at line 129. As noted below though I can remove this argument from ```inferIfNeeded()```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103116516
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSchemaInferenceSuite.scala ---
    @@ -0,0 +1,200 @@
    +/*
    + * 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.hive
    +
    +import java.io.File
    +import java.util.concurrent.{Executors, TimeUnit}
    +
    +import org.scalatest.BeforeAndAfterEach
    +
    +import org.apache.spark.metrics.source.HiveCatalogMetrics
    +import org.apache.spark.sql.catalyst.TableIdentifier
    +import org.apache.spark.sql.catalyst.catalog._
    +import org.apache.spark.sql.execution.datasources.FileStatusCache
    +import org.apache.spark.sql.QueryTest
    +import org.apache.spark.sql.hive.client.HiveClient
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +import org.apache.spark.sql.internal.SQLConf
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    +import org.apache.spark.sql.test.SQLTestUtils
    +import org.apache.spark.sql.types._
    +
    +class HiveSchemaInferenceSuite
    +  extends QueryTest with TestHiveSingleton with SQLTestUtils with BeforeAndAfterEach {
    +
    +  import HiveSchemaInferenceSuite._
    +  import HiveExternalCatalog.SPARK_SQL_PREFIX
    +
    +  override def beforeEach(): Unit = {
    +    super.beforeEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  override def afterEach(): Unit = {
    +    super.afterEach()
    +    FileStatusCache.resetForTesting()
    +  }
    +
    +  private val externalCatalog = spark.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog]
    +  private val lowercaseSchema = StructType(Seq(
    +    StructField("fieldone", LongType),
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +  private val caseSensitiveSchema = StructType(Seq(
    +    StructField("fieldOne", LongType),
    +    // Partition columns remain case-insensitive
    +    StructField("partcol1", IntegerType),
    +    StructField("partcol2", IntegerType)))
    +
    +  // Create a CatalogTable instance modeling an external Hive Metastore table backed by
    +  // Parquet data files.
    +  private def hiveExternalCatalogTable(
    +      tableName: String,
    +      location: String,
    +      schema: StructType,
    +      partitionColumns: Seq[String],
    +      properties: Map[String, String] = Map.empty): CatalogTable = {
    +    CatalogTable(
    +      identifier = TableIdentifier(table = tableName, database = Option(DATABASE)),
    +      tableType = CatalogTableType.EXTERNAL,
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(location),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")),
    +      schema = schema,
    +      provider = Option("hive"),
    +      partitionColumnNames = partitionColumns,
    +      properties = properties)
    +  }
    +
    +  // Creates CatalogTablePartition instances for adding partitions of data to our test table.
    +  private def hiveCatalogPartition(location: String, index: Int): CatalogTablePartition
    +    = CatalogTablePartition(
    +      spec = Map("partcol1" -> index.toString, "partcol2" -> index.toString),
    +      storage = CatalogStorageFormat(
    +        locationUri = Option(s"${location}/partCol1=$index/partCol2=$index/"),
    +        inputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"),
    +        outputFormat = Option("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"),
    +        serde = Option("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"),
    +        compressed = false,
    +        properties = Map("serialization.format" -> "1")))
    +
    +  // Creates a case-sensitive external Hive table for testing schema inference options. Table
    +  // will not have Spark-specific table properties set.
    +  private def setupCaseSensitiveTable(
    +      tableName: String,
    +      dir: File): Unit = {
    +    spark.range(NUM_RECORDS)
    +      .selectExpr("id as fieldOne", "id as partCol1", "id as partCol2")
    +      .write
    +      .partitionBy("partCol1", "partCol2")
    +      .mode("overwrite")
    +      .parquet(dir.getAbsolutePath)
    +
    +    val client = externalCatalog.client
    +
    +    val catalogTable = hiveExternalCatalogTable(
    +      tableName,
    +      dir.getAbsolutePath,
    +      lowercaseSchema,
    +      Seq("partcol1", "partcol2"))
    +    client.createTable(catalogTable, true)
    +
    +    val partitions = (0 until NUM_RECORDS).map(hiveCatalogPartition(dir.getAbsolutePath, _)).toSeq
    +    client.createPartitions(DATABASE, tableName, partitions, true)
    +
    +    // Check that the table returned by HiveExternalCatalog has schemaPreservesCase set to false
    +    // and that the raw table returned by the Hive client doesn't have and Spark SQL properties
    +    // set (table needs to be obtained from client since HiveExternalCatalog filters these
    +    // properties out).
    +    assert(!externalCatalog.getTable(DATABASE, TEST_TABLE_NAME).schemaPreservesCase)
    +    val rawTable = externalCatalog.client.getTable(DATABASE, TEST_TABLE_NAME)
    --- End diff --
    
    Making ```client``` a field of HiveSchemaInferenceSuite and replacing all instances of ```externalClient.catalog```


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Fixed conflicts introduced by #17015. I've also added support for schema inference on unpartitioned Hive tables as this was missing previously and really should be included here IMO. This led to refactoring ```HiveSchemaInferenceSuite``` so each test is run against partitioned and unpartitioned tables. ```HiveMetastoreCatalog.convertToLogicalRelation()``` has also been refactored quite a bit with the logic for performing schema inference moved to a new helper function.
    
    Pinging @cloud-fan, @viirya and @ericl to take a look.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Merged build finished. Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104537079
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +229,55 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(relation.tableMeta.location)
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = fileFormat
    +        .inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        .map(mergeWithMetastoreSchema(relation.tableMeta.schema, _))
    --- End diff --
    
    I'd still make the argument for including missing nullable fields in this as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104249753
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -159,6 +165,10 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               Some(partitionSchema))
     
             val logicalRelation = cached.getOrElse {
    +          val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    --- End diff --
    
    You mean just rename ```filteredMetastoreSchema``` to ```metastoreDataSchema```? Filtering out the partition columns from the schema will still be required. I agree that ```metastoreDataSchema``` is a better name though.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by cloud-fan <gi...@git.apache.org>.
Github user cloud-fan commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104750723
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -286,3 +347,33 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         }
       }
     }
    +
    +private[hive] object HiveMetastoreCatalog {
    --- End diff --
    
    nit: don't add `private[xxx]` if this class is already in "private" package. `org.apache.spark.sql.hive` is private.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r101462262
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,23 +161,49 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    +          val filteredMetastoreSchema = StructType(metastoreSchema
                   .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
     
    +          val inferenceMode = sparkSession.sessionState.conf.schemaInferenceMode
    +          val dataSchema = if (inferenceMode != "NEVER_INFER" &&
    +              !catalogTable.schemaFromTableProps) {
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    +            val merged = if (fileType.equals("parquet")) {
    +              inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +            } else {
    +              inferred
    --- End diff --
    
    Why don't we merge in the other cases?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r104271287
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -217,6 +227,62 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
         result.copy(expectedOutputAttributes = Some(relation.output))
       }
     
    +  private def inferIfNeeded(
    +      relation: CatalogRelation,
    +      options: Map[String, String],
    +      fallbackSchema: StructType,
    +      fileFormat: FileFormat,
    +      fileIndexOpt: Option[FileIndex] = None): (StructType, CatalogTable) = {
    +    val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +    val shouldInfer = (inferenceMode != NEVER_INFER) && !relation.tableMeta.schemaPreservesCase
    +    val tableName = relation.tableMeta.identifier.table
    +    if (shouldInfer) {
    +      logInfo(s"Inferring case-sensitive schema for table $tableName (inference mode: " +
    +        s"$inferenceMode)")
    +      val fileIndex = fileIndexOpt.getOrElse {
    +        val rootPath = new Path(new URI(relation.tableMeta.location))
    +        new InMemoryFileIndex(sparkSession, Seq(rootPath), options, None)
    +      }
    +
    +      val inferredSchema = {
    +        val schema = fileFormat.inferSchema(
    +          sparkSession,
    +          options,
    +          fileIndex.listFiles(Nil).flatMap(_.files))
    +        fileFormat match {
    +          case _: ParquetFileFormat =>
    +            schema.map(ParquetFileFormat.mergeMetastoreParquetSchema(relation.tableMeta.schema, _))
    +          case _ =>
    +            schema
    +        }
    +      }
    +
    +      inferredSchema match {
    +        case Some(schema) =>
    +          if (inferenceMode == INFER_AND_SAVE) {
    +            updateCatalogSchema(relation.tableMeta.identifier, schema)
    --- End diff --
    
    Thanks, I understand the issue with this now. I'll add code to merge the partition columns back in regardless of the file format. If we want to merge any nullable fields from the metastore schema that are missing from ```inferredSchema``` (see comment above) we probably want to do this explicitly as well.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    @viirya \u2013 Latest update incorporates your feedback. Thanks!
    
    @cloud-fan, @ericl \u2013 Any chance you could look at this updated PR?



---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by budde <gi...@git.apache.org>.
Github user budde commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r103049359
  
    --- Diff: sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfSuite.scala ---
    @@ -21,6 +21,7 @@ import org.apache.hadoop.fs.Path
     
     import org.apache.spark.sql._
     import org.apache.spark.sql.execution.WholeStageCodegenExec
    +import org.apache.spark.sql.internal.SQLConf.HiveCaseSensitiveInferenceMode
    --- End diff --
    
    Forgot to remove this when I removed the test in this file. Will fix.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark pull request #16944: [SPARK-19611][SQL] Introduce configurable table s...

Posted by ericl <gi...@git.apache.org>.
Github user ericl commented on a diff in the pull request:

    https://github.com/apache/spark/pull/16944#discussion_r102551354
  
    --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala ---
    @@ -161,22 +164,70 @@ private[hive] class HiveMetastoreCatalog(sparkSession: SparkSession) extends Log
               bucketSpec,
               Some(partitionSchema))
     
    +        val catalogTable = metastoreRelation.catalogTable
             val logicalRelation = cached.getOrElse {
               val sizeInBytes =
                 metastoreRelation.stats(sparkSession.sessionState.conf).sizeInBytes.toLong
               val fileIndex = {
    -            val index = new CatalogFileIndex(
    -              sparkSession, metastoreRelation.catalogTable, sizeInBytes)
    +            val index = new CatalogFileIndex(sparkSession, catalogTable, sizeInBytes)
                 if (lazyPruningEnabled) {
                   index
                 } else {
                   index.filterPartitions(Nil)  // materialize all the partitions in memory
                 }
               }
               val partitionSchemaColumnNames = partitionSchema.map(_.name.toLowerCase).toSet
    -          val dataSchema =
    -            StructType(metastoreSchema
    -              .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +          val filteredMetastoreSchema = StructType(metastoreSchema
    +            .filterNot(field => partitionSchemaColumnNames.contains(field.name.toLowerCase)))
    +
    +          // Infer a case-sensitive schema when the metastore doesn't return one, if configured.
    +          val inferenceMode = sparkSession.sessionState.conf.caseSensitiveInferenceMode
    +          val inferredSchema = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode != HiveCaseSensitiveInferenceMode.NEVER_INFER) {
    +            logInfo(s"Inferring case-sensitive schema for table $tableIdentifier")
    +            val fileStatuses = fileIndex.listFiles(Nil).flatMap(_.files)
    +            val inferred = defaultSource.inferSchema(sparkSession, options, fileStatuses)
    +            if (fileType.equals("parquet")) {
    +              inferred.map(ParquetFileFormat.mergeMetastoreParquetSchema(metastoreSchema, _))
    +            } else {
    +              inferred
    +            }
    +          } else {
    +            None
    +          }
    +
    +          // If configured, save the inferred case-sensitive schema to the table properties and
    +          // fetch the updated CatalogTable record for use in the LogicalRelation.
    +          val updatedCatalogTable = if (!catalogTable.schemaPreservesCase &&
    +              inferenceMode == HiveCaseSensitiveInferenceMode.INFER_AND_SAVE) {
    +            inferredSchema.flatMap { schema =>
    +              logInfo(s"Saving case-sensitive schema for table $tableIdentifier to table " +
    +                "properties")
    +              val updatedTable = catalogTable.copy(schema = schema)
    +              try {
    +                val catalog = sparkSession.sharedState.externalCatalog
    +                catalog.alterTable(updatedTable)
    +                Option(catalog.getTable(updatedTable.identifier.database.get,
    +                  updatedTable.identifier.table))
    +              } catch {
    +                case NonFatal(ex) =>
    +                  logError(s"Error saving case-sensitive schema for table $tableIdentifier: $ex")
    --- End diff --
    
    logWarn, also mention the config here?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73808 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73808/testReport)** for PR 16944 at commit [`514ae06`](https://github.com/apache/spark/commit/514ae06e1dbe2640091c90d55354c3500857e6e2).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #73863 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/73863/testReport)** for PR 16944 at commit [`2b2dbe8`](https://github.com/apache/spark/commit/2b2dbe8fbeea1b0a999303acb9358c9584d73b1b).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by SparkQA <gi...@git.apache.org>.
Github user SparkQA commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    **[Test build #74050 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74050/testReport)** for PR 16944 at commit [`95a70ce`](https://github.com/apache/spark/commit/95a70ce6443d3c2f9fefd9e473c48ecad641ba7b).


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org


[GitHub] spark issue #16944: [SPARK-19611][SQL] Introduce configurable table schema i...

Posted by AmplabJenkins <gi...@git.apache.org>.
Github user AmplabJenkins commented on the issue:

    https://github.com/apache/spark/pull/16944
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/73014/
    Test PASSed.


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

---------------------------------------------------------------------
To unsubscribe, e-mail: reviews-unsubscribe@spark.apache.org
For additional commands, e-mail: reviews-help@spark.apache.org