You are viewing a plain text version of this content. The canonical link for it is here.
Posted to reviews@spark.apache.org by windpiger <gi...@git.apache.org> on 2017/03/14 09:16:34 UTC

[GitHub] spark pull request #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

GitHub user windpiger opened a pull request:

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

    [SPARK-19945][SQL]add test suite for SessionCatalog with HiveExternalCatalog

    
    ## What changes were proposed in this pull request?
    
    Currently `SessionCatalogSuite` is only for `InMemoryCatalog`, there is no suite for `HiveExternalCatalog`.
    And there are some ddl function is not proper to test in `ExternalCatalogSuite`, because some logic are not full implement in `ExternalCatalog`, these ddl functions are full implement in `SessionCatalog`, it is better to test it in `SessionCatalogSuite`
    
    So we should add a test suite for `SessionCatalog` with `HiveExternalCatalog`
    ## How was this patch tested?
    add `HiveExternalSessionCatalogSuite`

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

    $ git pull https://github.com/windpiger/spark sessioncatalogsuit

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

    https://github.com/apache/spark/pull/17287.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 #17287
    
----
commit 44684461c86d2810ffdfae23485d74fba3801e12
Author: windpiger <so...@outlook.com>
Date:   2017-03-14T09:11:57Z

    [SPARK-19945][SQL]add test suite for SessionCatalog with 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r105860834
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1094,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    --- End diff --
    
    this is the expected part.
    https://github.com/apache/spark/blob/master/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/ExternalCatalogSuite.scala#L857
    
    here add some normalize logic


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74531/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74504 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74504/testReport)** for PR 17287 at commit [`d25466c`](https://github.com/apache/spark/commit/d25466c3632cd624daf3b4f3ea148db48d412252).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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/17287#discussion_r106093186
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    +
    +  private def withSessionCatalog(
    +      catalogTpe: String = BASIC)(f: SessionCatalog => Unit): Unit = {
    +    val catalog = catalogTpe match {
    +      case BASIC =>
    +        new SessionCatalog(newBasicCatalog())
    +      case EMPTY =>
    +        new SessionCatalog(newEmptyCatalog())
    +    }
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withSessionCatalogAndExternal(
    --- End diff --
    
    yea we should just access the external 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74637/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106020447
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -76,468 +118,499 @@ class SessionCatalogSuite extends PlanTest {
       }
     
       test("create databases using invalid names") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    testInvalidName(name => catalog.createDatabase(newDb(name), ignoreIfExists = true))
    +    withSessionCatalog(EMPTY) { catalog =>
    +      testInvalidName(
    +        name => catalog.createDatabase(newDb(name), ignoreIfExists = true))
    +    }
       }
     
       test("get database when a database exists") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    val db1 = catalog.getDatabaseMetadata("db1")
    -    assert(db1.name == "db1")
    -    assert(db1.description.contains("db1"))
    +    withSessionCatalog() { catalog =>
    +      val db1 = catalog.getDatabaseMetadata("db1")
    +      assert(db1.name == "db1")
    +      assert(db1.description.contains("db1"))
    +    }
       }
     
       test("get database should throw exception when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.getDatabaseMetadata("db_that_does_not_exist")
    +    withSessionCatalog() { catalog =>
    +      intercept[NoSuchDatabaseException] {
    +        catalog.getDatabaseMetadata("db_that_does_not_exist")
    +      }
         }
       }
     
       test("list databases without pattern") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.listDatabases().toSet == Set("default", "db1", "db2", "db3"))
    +    withSessionCatalog() { catalog =>
    +      assert(catalog.listDatabases().toSet == Set("default", "db1", "db2", "db3"))
    +    }
       }
     
       test("list databases with pattern") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.listDatabases("db").toSet == Set.empty)
    -    assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
    -    assert(catalog.listDatabases("*1").toSet == Set("db1"))
    -    assert(catalog.listDatabases("db2").toSet == Set("db2"))
    +    withSessionCatalog() { catalog =>
    +      assert(catalog.listDatabases("db").toSet == Set.empty)
    +      assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
    +      assert(catalog.listDatabases("*1").toSet == Set("db1"))
    +      assert(catalog.listDatabases("db2").toSet == Set("db2"))
    +    }
       }
     
       test("drop database") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false)
    -    assert(catalog.listDatabases().toSet == Set("default", "db2", "db3"))
    +    withSessionCatalog() { catalog =>
    +      catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false)
    +      assert(catalog.listDatabases().toSet == Set("default", "db2", "db3"))
    +    }
       }
     
       test("drop database when the database is not empty") {
         // Throw exception if there are functions left
    -    val externalCatalog1 = newBasicCatalog()
    -    val sessionCatalog1 = new SessionCatalog(externalCatalog1)
    -    externalCatalog1.dropTable("db2", "tbl1", ignoreIfNotExists = false, purge = false)
    -    externalCatalog1.dropTable("db2", "tbl2", ignoreIfNotExists = false, purge = false)
    -    intercept[AnalysisException] {
    -      sessionCatalog1.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +    withSessionCatalogAndExternal() { (catalog, externalCatalog) =>
    +      externalCatalog.dropTable("db2", "tbl1", ignoreIfNotExists = false, purge = false)
    +      externalCatalog.dropTable("db2", "tbl2", ignoreIfNotExists = false, purge = false)
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +      }
         }
    -
    -    // Throw exception if there are tables left
    -    val externalCatalog2 = newBasicCatalog()
    -    val sessionCatalog2 = new SessionCatalog(externalCatalog2)
    -    externalCatalog2.dropFunction("db2", "func1")
    -    intercept[AnalysisException] {
    -      sessionCatalog2.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +    withSessionCatalogAndExternal() { (catalog, externalCatalog) =>
    +      // Throw exception if there are tables left
    +      externalCatalog.dropFunction("db2", "func1")
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +      }
         }
     
    -    // When cascade is true, it should drop them
    -    val externalCatalog3 = newBasicCatalog()
    -    val sessionCatalog3 = new SessionCatalog(externalCatalog3)
    -    externalCatalog3.dropDatabase("db2", ignoreIfNotExists = false, cascade = true)
    -    assert(sessionCatalog3.listDatabases().toSet == Set("default", "db1", "db3"))
    +    withSessionCatalogAndExternal() { (catalog, externalCatalog) =>
    +      // When cascade is true, it should drop them
    +      externalCatalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = true)
    +      assert(catalog.listDatabases().toSet == Set("default", "db1", "db3"))
    +    }
       }
     
       test("drop database when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false)
    +    withSessionCatalog() { catalog =>
    +      if (isHiveExternalCatalog) {
    --- End diff --
    
    For any case like this. Please add a TODO for unifying them. 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 pull request #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106020015
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    +
    +  private def withSessionCatalog(
    +      catalogTpe: String = BASIC)(f: SessionCatalog => Unit): Unit = {
    +    val catalog = catalogTpe match {
    +      case BASIC =>
    +        new SessionCatalog(newBasicCatalog())
    +      case EMPTY =>
    +        new SessionCatalog(newEmptyCatalog())
    +    }
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withSessionCatalogAndExternal(
    --- End diff --
    
    cc @cloud-fan to confirm this solution is ok


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106333835
  
    --- Diff: sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveExternalSessionCatalogSuite.scala ---
    @@ -0,0 +1,40 @@
    +/*
    + * 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 org.apache.spark.sql.catalyst.catalog.{CatalogTestUtils, ExternalCatalog, SessionCatalogSuite}
    +import org.apache.spark.sql.hive.test.TestHiveSingleton
    +
    +class HiveExternalSessionCatalogSuite extends SessionCatalogSuite with TestHiveSingleton {
    +
    +  protected override val isHiveExternalCatalog = true
    +
    +  private val externalCatalog = {
    +    val catalog = spark.sharedState.externalCatalog
    +    catalog.asInstanceOf[HiveExternalCatalog].client.reset()
    +    catalog
    +  }
    +
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "org.apache.hadoop.mapred.SequenceFileInputFormat"
    +    override val tableOutputFormat: String =
    +      "org.apache.hadoop.hive.ql.io.HiveSequenceFileOutputFormat"
    +    override val defaultProvider: String = "parquet"
    --- End diff --
    
    The above input and output formats does not match what you specified here. Let us change it to `hive`


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74514/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74637 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74637/testReport)** for PR 17287 at commit [`80df8c7`](https://github.com/apache/spark/commit/80df8c74fc2280d9ca3d9fa2c6a624c6970ed6da).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74642 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74642/testReport)** for PR 17287 at commit [`25da5f6`](https://github.com/apache/spark/commit/25da5f6bfe99e1bf81856a353e7d572a8594a759).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74531 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74531/testReport)** for PR 17287 at commit [`47c1797`](https://github.com/apache/spark/commit/47c1797288677f63fc2124c949aac129346196d7).
     * This patch passes all tests.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HiveExternalCatalogSuite extends ExternalCatalogSuite `
      * `class HiveExternalSessionCatalogSuite extends SessionCatalogSuite with TestHiveSingleton `


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74596 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74596/testReport)** for PR 17287 at commit [`35c561f`](https://github.com/apache/spark/commit/35c561fe847c9d3df5f31f49f42e56061adb76c5).


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74638/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74647 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74647/testReport)** for PR 17287 at commit [`d82e8ed`](https://github.com/apache/spark/commit/d82e8eda4eed494604b131f1448fd93be3c1e33a).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74512 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74512/testReport)** for PR 17287 at commit [`c895a30`](https://github.com/apache/spark/commit/c895a30a5d3bb432d5208cb663e967fd7b606e30).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74643/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74532 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74532/testReport)** for PR 17287 at commit [`c360936`](https://github.com/apache/spark/commit/c360936a4a689083159e11ea3a0b5261265910dd).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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/17287#discussion_r106568913
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withEmptyCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newEmptyCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
       // --------------------------------------------------------------------------
       // Databases
       // --------------------------------------------------------------------------
     
       test("basic create and list databases") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    -    assert(catalog.databaseExists("default"))
    -    assert(!catalog.databaseExists("testing"))
    -    assert(!catalog.databaseExists("testing2"))
    -    catalog.createDatabase(newDb("testing"), ignoreIfExists = false)
    -    assert(catalog.databaseExists("testing"))
    -    assert(catalog.listDatabases().toSet == Set("default", "testing"))
    -    catalog.createDatabase(newDb("testing2"), ignoreIfExists = false)
    -    assert(catalog.listDatabases().toSet == Set("default", "testing", "testing2"))
    -    assert(catalog.databaseExists("testing2"))
    -    assert(!catalog.databaseExists("does_not_exist"))
    +    withEmptyCatalog { catalog =>
    +      catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    this is an example that we should not create default database in `withEmptyCatalog`, 


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106336045
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1083,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    +      p.copy(parameters = Map.empty, storage = p.storage.copy(
    +        properties = Map.empty, locationUri = None, serde = None))).toSet
    +
    +    val expectedPartsNormalize = expectedParts.map(p =>
    +        p.copy(parameters = Map.empty, storage = p.storage.copy(
    +          properties = Map.empty, locationUri = None, serde = None))).toSet
    +
    +    actualPartsNormalize == expectedPartsNormalize
    +//    actualParts.map(p =>
    +//      p.copy(storage = p.storage.copy(
    +//        properties = Map.empty, locationUri = None))).toSet ==
    +//      expectedParts.map(p =>
    +//        p.copy(storage = p.storage.copy(properties = Map.empty, locationUri = None))).toSet
    --- End diff --
    
    sorry, let me 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 issue #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74532/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106334626
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1083,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    +      p.copy(parameters = Map.empty, storage = p.storage.copy(
    +        properties = Map.empty, locationUri = None, serde = None))).toSet
    +
    +    val expectedPartsNormalize = expectedParts.map(p =>
    +        p.copy(parameters = Map.empty, storage = p.storage.copy(
    +          properties = Map.empty, locationUri = None, serde = None))).toSet
    +
    +    actualPartsNormalize == expectedPartsNormalize
    +//    actualParts.map(p =>
    +//      p.copy(storage = p.storage.copy(
    +//        properties = Map.empty, locationUri = None))).toSet ==
    +//      expectedParts.map(p =>
    +//        p.copy(storage = p.storage.copy(properties = Map.empty, locationUri = None))).toSet
    --- End diff --
    
    ?


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106065119
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -76,468 +118,499 @@ class SessionCatalogSuite extends PlanTest {
       }
     
       test("create databases using invalid names") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    testInvalidName(name => catalog.createDatabase(newDb(name), ignoreIfExists = true))
    +    withSessionCatalog(EMPTY) { catalog =>
    +      testInvalidName(
    +        name => catalog.createDatabase(newDb(name), ignoreIfExists = true))
    +    }
       }
     
       test("get database when a database exists") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    val db1 = catalog.getDatabaseMetadata("db1")
    -    assert(db1.name == "db1")
    -    assert(db1.description.contains("db1"))
    +    withSessionCatalog() { catalog =>
    +      val db1 = catalog.getDatabaseMetadata("db1")
    +      assert(db1.name == "db1")
    +      assert(db1.description.contains("db1"))
    +    }
       }
     
       test("get database should throw exception when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.getDatabaseMetadata("db_that_does_not_exist")
    +    withSessionCatalog() { catalog =>
    +      intercept[NoSuchDatabaseException] {
    +        catalog.getDatabaseMetadata("db_that_does_not_exist")
    +      }
         }
       }
     
       test("list databases without pattern") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.listDatabases().toSet == Set("default", "db1", "db2", "db3"))
    +    withSessionCatalog() { catalog =>
    +      assert(catalog.listDatabases().toSet == Set("default", "db1", "db2", "db3"))
    +    }
       }
     
       test("list databases with pattern") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.listDatabases("db").toSet == Set.empty)
    -    assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
    -    assert(catalog.listDatabases("*1").toSet == Set("db1"))
    -    assert(catalog.listDatabases("db2").toSet == Set("db2"))
    +    withSessionCatalog() { catalog =>
    +      assert(catalog.listDatabases("db").toSet == Set.empty)
    +      assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
    +      assert(catalog.listDatabases("*1").toSet == Set("db1"))
    +      assert(catalog.listDatabases("db2").toSet == Set("db2"))
    +    }
       }
     
       test("drop database") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false)
    -    assert(catalog.listDatabases().toSet == Set("default", "db2", "db3"))
    +    withSessionCatalog() { catalog =>
    +      catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false)
    +      assert(catalog.listDatabases().toSet == Set("default", "db2", "db3"))
    +    }
       }
     
       test("drop database when the database is not empty") {
         // Throw exception if there are functions left
    -    val externalCatalog1 = newBasicCatalog()
    -    val sessionCatalog1 = new SessionCatalog(externalCatalog1)
    -    externalCatalog1.dropTable("db2", "tbl1", ignoreIfNotExists = false, purge = false)
    -    externalCatalog1.dropTable("db2", "tbl2", ignoreIfNotExists = false, purge = false)
    -    intercept[AnalysisException] {
    -      sessionCatalog1.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +    withSessionCatalogAndExternal() { (catalog, externalCatalog) =>
    +      externalCatalog.dropTable("db2", "tbl1", ignoreIfNotExists = false, purge = false)
    +      externalCatalog.dropTable("db2", "tbl2", ignoreIfNotExists = false, purge = false)
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +      }
         }
    -
    -    // Throw exception if there are tables left
    -    val externalCatalog2 = newBasicCatalog()
    -    val sessionCatalog2 = new SessionCatalog(externalCatalog2)
    -    externalCatalog2.dropFunction("db2", "func1")
    -    intercept[AnalysisException] {
    -      sessionCatalog2.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +    withSessionCatalogAndExternal() { (catalog, externalCatalog) =>
    +      // Throw exception if there are tables left
    +      externalCatalog.dropFunction("db2", "func1")
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +      }
         }
     
    -    // When cascade is true, it should drop them
    -    val externalCatalog3 = newBasicCatalog()
    -    val sessionCatalog3 = new SessionCatalog(externalCatalog3)
    -    externalCatalog3.dropDatabase("db2", ignoreIfNotExists = false, cascade = true)
    -    assert(sessionCatalog3.listDatabases().toSet == Set("default", "db1", "db3"))
    +    withSessionCatalogAndExternal() { (catalog, externalCatalog) =>
    +      // When cascade is true, it should drop them
    +      externalCatalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = true)
    +      assert(catalog.listDatabases().toSet == Set("default", "db1", "db3"))
    +    }
       }
     
       test("drop database when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false)
    +    withSessionCatalog() { catalog =>
    +      if (isHiveExternalCatalog) {
    --- End diff --
    
    ok ~ 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106145746
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    +
    +  private def withSessionCatalog(
    --- End diff --
    
    ok~ 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 pull request #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106569991
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withEmptyCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newEmptyCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
       // --------------------------------------------------------------------------
       // Databases
       // --------------------------------------------------------------------------
     
       test("basic create and list databases") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    -    assert(catalog.databaseExists("default"))
    -    assert(!catalog.databaseExists("testing"))
    -    assert(!catalog.databaseExists("testing2"))
    -    catalog.createDatabase(newDb("testing"), ignoreIfExists = false)
    -    assert(catalog.databaseExists("testing"))
    -    assert(catalog.listDatabases().toSet == Set("default", "testing"))
    -    catalog.createDatabase(newDb("testing2"), ignoreIfExists = false)
    -    assert(catalog.listDatabases().toSet == Set("default", "testing", "testing2"))
    -    assert(catalog.databaseExists("testing2"))
    -    assert(!catalog.databaseExists("does_not_exist"))
    +    withEmptyCatalog { catalog =>
    +      catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    here the ignoreIfExists = true, I think it is ok that the default database have already created.


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74514 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74514/testReport)** for PR 17287 at commit [`aaaedf9`](https://github.com/apache/spark/commit/aaaedf9e061f36e3f599c2abc159cab5e655784f).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106340900
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1270,6 +1376,7 @@ class SessionCatalogSuite extends PlanTest {
           }
     
           assert(cause.getMessage.contains("Undefined function: 'undefined_fn'"))
    +      catalog.reset()
    --- End diff --
    
    Then, this `reset()` could be skipped if hitting an exception.


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106334485
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1270,6 +1376,7 @@ class SessionCatalogSuite extends PlanTest {
           }
     
           assert(cause.getMessage.contains("Undefined function: 'undefined_fn'"))
    +      catalog.reset()
    --- End diff --
    
    Instead of adding `reset`, why not using your new function `withBasicCatalog`?


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106334827
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1094,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    --- End diff --
    
    Because Hive metastore fills the values after we call the Hive APIs?


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106570569
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withEmptyCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newEmptyCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
       // --------------------------------------------------------------------------
       // Databases
       // --------------------------------------------------------------------------
     
       test("basic create and list databases") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    -    assert(catalog.databaseExists("default"))
    -    assert(!catalog.databaseExists("testing"))
    -    assert(!catalog.databaseExists("testing2"))
    -    catalog.createDatabase(newDb("testing"), ignoreIfExists = false)
    -    assert(catalog.databaseExists("testing"))
    -    assert(catalog.listDatabases().toSet == Set("default", "testing"))
    -    catalog.createDatabase(newDb("testing2"), ignoreIfExists = false)
    -    assert(catalog.listDatabases().toSet == Set("default", "testing", "testing2"))
    -    assert(catalog.databaseExists("testing2"))
    -    assert(!catalog.databaseExists("does_not_exist"))
    +    withEmptyCatalog { catalog =>
    +      catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    This is a basic test case to test `create and list databases`. Thus, it will be good to create a database without the existing 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74638 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74638/testReport)** for PR 17287 at commit [`4214379`](https://github.com/apache/spark/commit/421437951df5d3bb551dc62428bbd3c23cd94f4e).


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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/17287#discussion_r106093083
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    +
    +  private def withSessionCatalog(
    --- End diff --
    
    I'd like to have 2 methods: `withBasicCatalog` and `withEmptyCatalog`


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74637 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74637/testReport)** for PR 17287 at commit [`80df8c7`](https://github.com/apache/spark/commit/80df8c74fc2280d9ca3d9fa2c6a624c6970ed6da).


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r105858890
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    +
    +  private def withSessionCatalog(
    +      catalogTpe: String = BASIC)(f: SessionCatalog => Unit): Unit = {
    +    val catalog = catalogTpe match {
    +      case BASIC =>
    +        new SessionCatalog(newBasicCatalog())
    +      case EMPTY =>
    +        new SessionCatalog(newEmptyCatalog())
    +    }
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    --- End diff --
    
    reset the 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74502 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74502/testReport)** for PR 17287 at commit [`9ee9fa7`](https://github.com/apache/spark/commit/9ee9fa77d988e79e4d7f3cbf5f9708d7848ebd70).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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/17287#discussion_r106568853
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withEmptyCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newEmptyCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    shall we do this? Previously when we call `newEmptyCatalog` in test cases, there is no default database created


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    I addressed these comments in the PR https://github.com/apache/spark/pull/17354


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74638 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74638/testReport)** for PR 17287 at commit [`4214379`](https://github.com/apache/spark/commit/421437951df5d3bb551dc62428bbd3c23cd94f4e).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74642 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74642/testReport)** for PR 17287 at commit [`25da5f6`](https://github.com/apache/spark/commit/25da5f6bfe99e1bf81856a353e7d572a8594a759).


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Thanks! Merging to master.


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74596 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74596/testReport)** for PR 17287 at commit [`35c561f`](https://github.com/apache/spark/commit/35c561fe847c9d3df5f31f49f42e56061adb76c5).
     * 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74521/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106017976
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    --- End diff --
    
    These should be moved to `CatalogTestUtils`


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    @gatorsmile @cloud-fan I have fix some code view, could you help to continue reivew 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 pull request #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106340219
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1083,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    --- End diff --
    
    You need to leave a comment to explain 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74512/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74502/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106019799
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,83 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  val BASIC = "basic"
    +  val EMPTY = "empty"
    +
    +  private def withSessionCatalog(
    +      catalogTpe: String = BASIC)(f: SessionCatalog => Unit): Unit = {
    +    val catalog = catalogTpe match {
    +      case BASIC =>
    +        new SessionCatalog(newBasicCatalog())
    +      case EMPTY =>
    +        new SessionCatalog(newEmptyCatalog())
    +    }
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withSessionCatalogAndExternal(
    --- End diff --
    
    This function is not needed, I think. 
    
    We just need to make ExternalCatalog accessible. Then, the test case can directly access it by `catalog.externalCatalog`
    ```
    class SessionCatalog(
        val externalCatalog: ExternalCatalog,
        globalTempViewManager: GlobalTempViewManager,
        functionRegistry: FunctionRegistry,
        conf: CatalystConf,
        hadoopConf: Configuration,
        parser: ParserInterface) 
    ```


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106342082
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1270,6 +1376,7 @@ class SessionCatalogSuite extends PlanTest {
           }
     
           assert(cause.getMessage.contains("Undefined function: 'undefined_fn'"))
    +      catalog.reset()
    --- End diff --
    
    yes, you are right, let me add a try catch


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106335778
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1083,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    --- End diff --
    
    Yes, it is~


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74501 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74501/testReport)** for PR 17287 at commit [`4468446`](https://github.com/apache/spark/commit/44684461c86d2810ffdfae23485d74fba3801e12).
     * This patch **fails Scala style tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class InMemorySessionCatalogSuite extends SessionCatalogSuite `
      * `abstract class SessionCatalogSuite extends PlanTest `
      * `class HiveExternalSessionCatalogSuite extends SessionCatalogSuite `


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74501 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74501/testReport)** for PR 17287 at commit [`4468446`](https://github.com/apache/spark/commit/44684461c86d2810ffdfae23485d74fba3801e12).


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74504/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 issue #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74531 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74531/testReport)** for PR 17287 at commit [`47c1797`](https://github.com/apache/spark/commit/47c1797288677f63fc2124c949aac129346196d7).


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    **[Test build #74521 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/74521/testReport)** for PR 17287 at commit [`bc8af7e`](https://github.com/apache/spark/commit/bc8af7e9eaf6f52ebae5e9c07660ce37dae26c34).
     * This patch **fails Spark unit tests**.
     * This patch merges cleanly.
     * This patch adds the following public classes _(experimental)_:
      * `class HiveExternalCatalogSuite extends ExternalCatalogSuite with BeforeAndAfterAll `


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    a late 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106334939
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -999,257 +1083,279 @@ class SessionCatalogSuite extends PlanTest {
           expectedParts: CatalogTablePartition*): Boolean = {
         // ExternalCatalog may set a default location for partitions, here we ignore the partition
         // location when comparing them.
    -    actualParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet ==
    -      expectedParts.map(p => p.copy(storage = p.storage.copy(locationUri = None))).toSet
    +    val actualPartsNormalize = actualParts.map(p =>
    --- End diff --
    
    Because Hive metastore fills the values after we calling the Hive APIs? 


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106335967
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -1270,6 +1376,7 @@ class SessionCatalogSuite extends PlanTest {
           }
     
           assert(cause.getMessage.contains("Undefined function: 'undefined_fn'"))
    +      catalog.reset()
    --- End diff --
    
    here the `SessionCatalog` is instanced with different `conf` parameter.
    In `withBasicCatalog`, it just leave it default


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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/17287#discussion_r106568674
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    we already created the default database in `newBasicCatalog`


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106997366
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withEmptyCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newEmptyCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    Our `reset()` function assumes the existence of Default


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106336735
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -76,468 +102,500 @@ class SessionCatalogSuite extends PlanTest {
       }
     
       test("create databases using invalid names") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    testInvalidName(name => catalog.createDatabase(newDb(name), ignoreIfExists = true))
    +    withEmptyCatalog { catalog =>
    +      testInvalidName(
    +        name => catalog.createDatabase(newDb(name), ignoreIfExists = true))
    +    }
       }
     
       test("get database when a database exists") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    val db1 = catalog.getDatabaseMetadata("db1")
    -    assert(db1.name == "db1")
    -    assert(db1.description.contains("db1"))
    +    withBasicCatalog { catalog =>
    +      val db1 = catalog.getDatabaseMetadata("db1")
    +      assert(db1.name == "db1")
    +      assert(db1.description.contains("db1"))
    +    }
       }
     
       test("get database should throw exception when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.getDatabaseMetadata("db_that_does_not_exist")
    +    withBasicCatalog { catalog =>
    +      intercept[NoSuchDatabaseException] {
    +        catalog.getDatabaseMetadata("db_that_does_not_exist")
    +      }
         }
       }
     
       test("list databases without pattern") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.listDatabases().toSet == Set("default", "db1", "db2", "db3"))
    +    withBasicCatalog { catalog =>
    +      assert(catalog.listDatabases().toSet == Set("default", "db1", "db2", "db3"))
    +    }
       }
     
       test("list databases with pattern") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.listDatabases("db").toSet == Set.empty)
    -    assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
    -    assert(catalog.listDatabases("*1").toSet == Set("db1"))
    -    assert(catalog.listDatabases("db2").toSet == Set("db2"))
    +    withBasicCatalog { catalog =>
    +      assert(catalog.listDatabases("db").toSet == Set.empty)
    +      assert(catalog.listDatabases("db*").toSet == Set("db1", "db2", "db3"))
    +      assert(catalog.listDatabases("*1").toSet == Set("db1"))
    +      assert(catalog.listDatabases("db2").toSet == Set("db2"))
    +    }
       }
     
       test("drop database") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false)
    -    assert(catalog.listDatabases().toSet == Set("default", "db2", "db3"))
    +    withBasicCatalog { catalog =>
    +      catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = false)
    +      assert(catalog.listDatabases().toSet == Set("default", "db2", "db3"))
    +    }
       }
     
       test("drop database when the database is not empty") {
         // Throw exception if there are functions left
    -    val externalCatalog1 = newBasicCatalog()
    -    val sessionCatalog1 = new SessionCatalog(externalCatalog1)
    -    externalCatalog1.dropTable("db2", "tbl1", ignoreIfNotExists = false, purge = false)
    -    externalCatalog1.dropTable("db2", "tbl2", ignoreIfNotExists = false, purge = false)
    -    intercept[AnalysisException] {
    -      sessionCatalog1.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +    withBasicCatalog { catalog =>
    +      catalog.externalCatalog.dropTable("db2", "tbl1", ignoreIfNotExists = false, purge = false)
    +      catalog.externalCatalog.dropTable("db2", "tbl2", ignoreIfNotExists = false, purge = false)
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +      }
         }
    -
    -    // Throw exception if there are tables left
    -    val externalCatalog2 = newBasicCatalog()
    -    val sessionCatalog2 = new SessionCatalog(externalCatalog2)
    -    externalCatalog2.dropFunction("db2", "func1")
    -    intercept[AnalysisException] {
    -      sessionCatalog2.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +    withBasicCatalog { catalog =>
    +      // Throw exception if there are tables left
    +      catalog.externalCatalog.dropFunction("db2", "func1")
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = false)
    +      }
         }
     
    -    // When cascade is true, it should drop them
    -    val externalCatalog3 = newBasicCatalog()
    -    val sessionCatalog3 = new SessionCatalog(externalCatalog3)
    -    externalCatalog3.dropDatabase("db2", ignoreIfNotExists = false, cascade = true)
    -    assert(sessionCatalog3.listDatabases().toSet == Set("default", "db1", "db3"))
    +    withBasicCatalog { catalog =>
    +      // When cascade is true, it should drop them
    +      catalog.externalCatalog.dropDatabase("db2", ignoreIfNotExists = false, cascade = true)
    +      assert(catalog.listDatabases().toSet == Set("default", "db1", "db3"))
    +    }
       }
     
       test("drop database when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false)
    +    withBasicCatalog { catalog =>
    +      // TODO: fix this inconsistent between HiveExternalCatalog and InMemoryCatalog
    +      if (isHiveExternalCatalog) {
    +        val e = intercept[AnalysisException] {
    +          catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false)
    +        }.getMessage
    +        assert(e.contains(
    +          "org.apache.hadoop.hive.metastore.api.NoSuchObjectException: db_that_does_not_exist"))
    +      } else {
    +        intercept[NoSuchDatabaseException] {
    +          catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = false, cascade = false)
    +        }
    +      }
    +      catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = true, cascade = false)
         }
    -    catalog.dropDatabase("db_that_does_not_exist", ignoreIfNotExists = true, cascade = false)
       }
     
       test("drop current database and drop default database") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    catalog.setCurrentDatabase("db1")
    -    assert(catalog.getCurrentDatabase == "db1")
    -    catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = true)
    -    intercept[NoSuchDatabaseException] {
    -      catalog.createTable(newTable("tbl1", "db1"), ignoreIfExists = false)
    -    }
    -    catalog.setCurrentDatabase("default")
    -    assert(catalog.getCurrentDatabase == "default")
    -    intercept[AnalysisException] {
    -      catalog.dropDatabase("default", ignoreIfNotExists = false, cascade = true)
    +    withBasicCatalog { catalog =>
    +      catalog.setCurrentDatabase("db1")
    +      assert(catalog.getCurrentDatabase == "db1")
    +      catalog.dropDatabase("db1", ignoreIfNotExists = false, cascade = true)
    +      intercept[NoSuchDatabaseException] {
    +        catalog.createTable(newTable("tbl1", "db1"), ignoreIfExists = false)
    +      }
    +      catalog.setCurrentDatabase("default")
    +      assert(catalog.getCurrentDatabase == "default")
    +      intercept[AnalysisException] {
    +        catalog.dropDatabase("default", ignoreIfNotExists = false, cascade = true)
    +      }
         }
       }
     
       test("alter database") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    val db1 = catalog.getDatabaseMetadata("db1")
    -    // Note: alter properties here because Hive does not support altering other fields
    -    catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true")))
    -    val newDb1 = catalog.getDatabaseMetadata("db1")
    -    assert(db1.properties.isEmpty)
    -    assert(newDb1.properties.size == 2)
    -    assert(newDb1.properties.get("k") == Some("v3"))
    -    assert(newDb1.properties.get("good") == Some("true"))
    +    withBasicCatalog { catalog =>
    +      val db1 = catalog.getDatabaseMetadata("db1")
    +      // Note: alter properties here because Hive does not support altering other fields
    +      catalog.alterDatabase(db1.copy(properties = Map("k" -> "v3", "good" -> "true")))
    +      val newDb1 = catalog.getDatabaseMetadata("db1")
    +      assert(db1.properties.isEmpty)
    +      assert(newDb1.properties.size == 2)
    +      assert(newDb1.properties.get("k") == Some("v3"))
    +      assert(newDb1.properties.get("good") == Some("true"))
    +    }
       }
     
       test("alter database should throw exception when the database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.alterDatabase(newDb("unknown_db"))
    +    withBasicCatalog { catalog =>
    +      intercept[NoSuchDatabaseException] {
    +        catalog.alterDatabase(newDb("unknown_db"))
    +      }
         }
       }
     
       test("get/set current database") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    assert(catalog.getCurrentDatabase == "default")
    -    catalog.setCurrentDatabase("db2")
    -    assert(catalog.getCurrentDatabase == "db2")
    -    intercept[NoSuchDatabaseException] {
    +    withBasicCatalog { catalog =>
    +      assert(catalog.getCurrentDatabase == "default")
    +      catalog.setCurrentDatabase("db2")
    +      assert(catalog.getCurrentDatabase == "db2")
    +      intercept[NoSuchDatabaseException] {
    +        catalog.setCurrentDatabase("deebo")
    +      }
    +      catalog.createDatabase(newDb("deebo"), ignoreIfExists = false)
           catalog.setCurrentDatabase("deebo")
    +      assert(catalog.getCurrentDatabase == "deebo")
         }
    -    catalog.createDatabase(newDb("deebo"), ignoreIfExists = false)
    -    catalog.setCurrentDatabase("deebo")
    -    assert(catalog.getCurrentDatabase == "deebo")
       }
     
       // --------------------------------------------------------------------------
       // Tables
       // --------------------------------------------------------------------------
     
       test("create table") {
    -    val externalCatalog = newBasicCatalog()
    -    val sessionCatalog = new SessionCatalog(externalCatalog)
    -    assert(externalCatalog.listTables("db1").isEmpty)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    sessionCatalog.createTable(newTable("tbl3", "db1"), ignoreIfExists = false)
    -    sessionCatalog.createTable(newTable("tbl3", "db2"), ignoreIfExists = false)
    -    assert(externalCatalog.listTables("db1").toSet == Set("tbl3"))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3"))
    -    // Create table without explicitly specifying database
    -    sessionCatalog.setCurrentDatabase("db1")
    -    sessionCatalog.createTable(newTable("tbl4"), ignoreIfExists = false)
    -    assert(externalCatalog.listTables("db1").toSet == Set("tbl3", "tbl4"))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3"))
    +    withBasicCatalog { catalog =>
    +      assert(catalog.externalCatalog.listTables("db1").isEmpty)
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      catalog.createTable(newTable("tbl3", "db1"), ignoreIfExists = false)
    +      catalog.createTable(newTable("tbl3", "db2"), ignoreIfExists = false)
    +      assert(catalog.externalCatalog.listTables("db1").toSet == Set("tbl3"))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3"))
    +      // Create table without explicitly specifying database
    +      catalog.setCurrentDatabase("db1")
    +      catalog.createTable(newTable("tbl4"), ignoreIfExists = false)
    +      assert(catalog.externalCatalog.listTables("db1").toSet == Set("tbl3", "tbl4"))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2", "tbl3"))
    +    }
       }
     
       test("create tables using invalid names") {
    -    val catalog = new SessionCatalog(newEmptyCatalog())
    -    testInvalidName(name => catalog.createTable(newTable(name, "db1"), ignoreIfExists = false))
    +    withEmptyCatalog { catalog =>
    +      testInvalidName(name => catalog.createTable(newTable(name, "db1"), ignoreIfExists = false))
    +    }
       }
     
       test("create table when database does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    // Creating table in non-existent database should always fail
    -    intercept[NoSuchDatabaseException] {
    -      catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = false)
    -    }
    -    intercept[NoSuchDatabaseException] {
    -      catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = true)
    -    }
    -    // Table already exists
    -    intercept[TableAlreadyExistsException] {
    -      catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false)
    +    withBasicCatalog { catalog =>
    +      // Creating table in non-existent database should always fail
    +      intercept[NoSuchDatabaseException] {
    +        catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = false)
    +      }
    +      intercept[NoSuchDatabaseException] {
    +        catalog.createTable(newTable("tbl1", "does_not_exist"), ignoreIfExists = true)
    +      }
    +      // Table already exists
    +      intercept[TableAlreadyExistsException] {
    +        catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false)
    +      }
    +      catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = true)
         }
    -    catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = true)
       }
     
       test("create temp table") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    val tempTable1 = Range(1, 10, 1, 10)
    -    val tempTable2 = Range(1, 20, 2, 10)
    -    catalog.createTempView("tbl1", tempTable1, overrideIfExists = false)
    -    catalog.createTempView("tbl2", tempTable2, overrideIfExists = false)
    -    assert(catalog.getTempView("tbl1") == Option(tempTable1))
    -    assert(catalog.getTempView("tbl2") == Option(tempTable2))
    -    assert(catalog.getTempView("tbl3").isEmpty)
    -    // Temporary table already exists
    -    intercept[TempTableAlreadyExistsException] {
    +    withBasicCatalog { catalog =>
    +      val tempTable1 = Range(1, 10, 1, 10)
    +      val tempTable2 = Range(1, 20, 2, 10)
           catalog.createTempView("tbl1", tempTable1, overrideIfExists = false)
    +      catalog.createTempView("tbl2", tempTable2, overrideIfExists = false)
    +      assert(catalog.getTempView("tbl1") == Option(tempTable1))
    +      assert(catalog.getTempView("tbl2") == Option(tempTable2))
    +      assert(catalog.getTempView("tbl3").isEmpty)
    +      // Temporary table already exists
    +      intercept[TempTableAlreadyExistsException] {
    +        catalog.createTempView("tbl1", tempTable1, overrideIfExists = false)
    +      }
    +      // Temporary table already exists but we override it
    +      catalog.createTempView("tbl1", tempTable2, overrideIfExists = true)
    +      assert(catalog.getTempView("tbl1") == Option(tempTable2))
         }
    -    // Temporary table already exists but we override it
    -    catalog.createTempView("tbl1", tempTable2, overrideIfExists = true)
    -    assert(catalog.getTempView("tbl1") == Option(tempTable2))
       }
     
       test("drop table") {
    -    val externalCatalog = newBasicCatalog()
    -    val sessionCatalog = new SessionCatalog(externalCatalog)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false,
    -      purge = false)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl2"))
    -    // Drop table without explicitly specifying database
    -    sessionCatalog.setCurrentDatabase("db2")
    -    sessionCatalog.dropTable(TableIdentifier("tbl2"), ignoreIfNotExists = false, purge = false)
    -    assert(externalCatalog.listTables("db2").isEmpty)
    +    withBasicCatalog { catalog =>
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      catalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false,
    +        purge = false)
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl2"))
    +      // Drop table without explicitly specifying database
    +      catalog.setCurrentDatabase("db2")
    +      catalog.dropTable(TableIdentifier("tbl2"), ignoreIfNotExists = false, purge = false)
    +      assert(catalog.externalCatalog.listTables("db2").isEmpty)
    +    }
       }
     
       test("drop table when database/table does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    // Should always throw exception when the database does not exist
    -    intercept[NoSuchDatabaseException] {
    -      catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = false,
    -        purge = false)
    -    }
    -    intercept[NoSuchDatabaseException] {
    -      catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true,
    -        purge = false)
    -    }
    -    intercept[NoSuchTableException] {
    -      catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false,
    +    withBasicCatalog { catalog =>
    +      // Should always throw exception when the database does not exist
    +      intercept[NoSuchDatabaseException] {
    +        catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = false,
    +          purge = false)
    +      }
    +      intercept[NoSuchDatabaseException] {
    +        catalog.dropTable(TableIdentifier("tbl1", Some("unknown_db")), ignoreIfNotExists = true,
    +          purge = false)
    +      }
    +      intercept[NoSuchTableException] {
    +        catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = false,
    +          purge = false)
    +      }
    +      catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = true,
             purge = false)
         }
    -    catalog.dropTable(TableIdentifier("unknown_table", Some("db2")), ignoreIfNotExists = true,
    -      purge = false)
       }
     
       test("drop temp table") {
    -    val externalCatalog = newBasicCatalog()
    -    val sessionCatalog = new SessionCatalog(externalCatalog)
    -    val tempTable = Range(1, 10, 2, 10)
    -    sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false)
    -    sessionCatalog.setCurrentDatabase("db2")
    -    assert(sessionCatalog.getTempView("tbl1") == Some(tempTable))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    // If database is not specified, temp table should be dropped first
    -    sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false)
    -    assert(sessionCatalog.getTempView("tbl1") == None)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    // If temp table does not exist, the table in the current database should be dropped
    -    sessionCatalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl2"))
    -    // If database is specified, temp tables are never dropped
    -    sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false)
    -    sessionCatalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false)
    -    sessionCatalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false,
    -      purge = false)
    -    assert(sessionCatalog.getTempView("tbl1") == Some(tempTable))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl2"))
    +    withBasicCatalog { catalog =>
    +      val tempTable = Range(1, 10, 2, 10)
    +      catalog.createTempView("tbl1", tempTable, overrideIfExists = false)
    +      catalog.setCurrentDatabase("db2")
    +      assert(catalog.getTempView("tbl1") == Some(tempTable))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      // If database is not specified, temp table should be dropped first
    +      catalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false)
    +      assert(catalog.getTempView("tbl1") == None)
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      // If temp table does not exist, the table in the current database should be dropped
    +      catalog.dropTable(TableIdentifier("tbl1"), ignoreIfNotExists = false, purge = false)
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl2"))
    +      // If database is specified, temp tables are never dropped
    +      catalog.createTempView("tbl1", tempTable, overrideIfExists = false)
    +      catalog.createTable(newTable("tbl1", "db2"), ignoreIfExists = false)
    +      catalog.dropTable(TableIdentifier("tbl1", Some("db2")), ignoreIfNotExists = false,
    +        purge = false)
    +      assert(catalog.getTempView("tbl1") == Some(tempTable))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl2"))
    +    }
       }
     
       test("rename table") {
    -    val externalCatalog = newBasicCatalog()
    -    val sessionCatalog = new SessionCatalog(externalCatalog)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    sessionCatalog.renameTable(TableIdentifier("tbl1", Some("db2")), TableIdentifier("tblone"))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbl2"))
    -    sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo"))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo"))
    -    // Rename table without explicitly specifying database
    -    sessionCatalog.setCurrentDatabase("db2")
    -    sessionCatalog.renameTable(TableIdentifier("tbltwo"), TableIdentifier("table_two"))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tblone", "table_two"))
    -    // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match
    -    intercept[AnalysisException] {
    -      sessionCatalog.renameTable(
    -        TableIdentifier("tblone", Some("db2")), TableIdentifier("tblones", Some("db1")))
    -    }
    -    // The new table already exists
    -    intercept[TableAlreadyExistsException] {
    -      sessionCatalog.renameTable(
    -        TableIdentifier("tblone", Some("db2")),
    -        TableIdentifier("table_two"))
    +    withBasicCatalog { catalog =>
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      catalog.renameTable(TableIdentifier("tbl1", Some("db2")), TableIdentifier("tblone"))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tblone", "tbl2"))
    +      catalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbltwo"))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tblone", "tbltwo"))
    +      // Rename table without explicitly specifying database
    +      catalog.setCurrentDatabase("db2")
    +      catalog.renameTable(TableIdentifier("tbltwo"), TableIdentifier("table_two"))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tblone", "table_two"))
    +      // Renaming "db2.tblone" to "db1.tblones" should fail because databases don't match
    +      intercept[AnalysisException] {
    +        catalog.renameTable(
    +          TableIdentifier("tblone", Some("db2")), TableIdentifier("tblones", Some("db1")))
    +      }
    +      // The new table already exists
    +      intercept[TableAlreadyExistsException] {
    +        catalog.renameTable(
    +          TableIdentifier("tblone", Some("db2")),
    +          TableIdentifier("table_two"))
    +      }
         }
       }
     
       test("rename tables to an invalid name") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    testInvalidName(
    -      name => catalog.renameTable(TableIdentifier("tbl1", Some("db2")), TableIdentifier(name)))
    +    withBasicCatalog { catalog =>
    +      testInvalidName(
    +        name => catalog.renameTable(TableIdentifier("tbl1", Some("db2")), TableIdentifier(name)))
    +    }
       }
     
       test("rename table when database/table does not exist") {
    -    val catalog = new SessionCatalog(newBasicCatalog())
    -    intercept[NoSuchDatabaseException] {
    -      catalog.renameTable(TableIdentifier("tbl1", Some("unknown_db")), TableIdentifier("tbl2"))
    -    }
    -    intercept[NoSuchTableException] {
    -      catalog.renameTable(TableIdentifier("unknown_table", Some("db2")), TableIdentifier("tbl2"))
    +    withBasicCatalog { catalog =>
    +      intercept[NoSuchDatabaseException] {
    +        catalog.renameTable(TableIdentifier("tbl1", Some("unknown_db")), TableIdentifier("tbl2"))
    +      }
    +      intercept[NoSuchTableException] {
    +        catalog.renameTable(TableIdentifier("unknown_table", Some("db2")), TableIdentifier("tbl2"))
    +      }
         }
       }
     
       test("rename temp table") {
    -    val externalCatalog = newBasicCatalog()
    -    val sessionCatalog = new SessionCatalog(externalCatalog)
    -    val tempTable = Range(1, 10, 2, 10)
    -    sessionCatalog.createTempView("tbl1", tempTable, overrideIfExists = false)
    -    sessionCatalog.setCurrentDatabase("db2")
    -    assert(sessionCatalog.getTempView("tbl1") == Option(tempTable))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    // If database is not specified, temp table should be renamed first
    -    sessionCatalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3"))
    -    assert(sessionCatalog.getTempView("tbl1").isEmpty)
    -    assert(sessionCatalog.getTempView("tbl3") == Option(tempTable))
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    -    // If database is specified, temp tables are never renamed
    -    sessionCatalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4"))
    -    assert(sessionCatalog.getTempView("tbl3") == Option(tempTable))
    -    assert(sessionCatalog.getTempView("tbl4").isEmpty)
    -    assert(externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4"))
    +    withBasicCatalog { catalog =>
    +      val tempTable = Range(1, 10, 2, 10)
    +      catalog.createTempView("tbl1", tempTable, overrideIfExists = false)
    +      catalog.setCurrentDatabase("db2")
    +      assert(catalog.getTempView("tbl1") == Option(tempTable))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      // If database is not specified, temp table should be renamed first
    +      catalog.renameTable(TableIdentifier("tbl1"), TableIdentifier("tbl3"))
    +      assert(catalog.getTempView("tbl1").isEmpty)
    +      assert(catalog.getTempView("tbl3") == Option(tempTable))
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl2"))
    +      // If database is specified, temp tables are never renamed
    +      catalog.renameTable(TableIdentifier("tbl2", Some("db2")), TableIdentifier("tbl4"))
    +      assert(catalog.getTempView("tbl3") == Option(tempTable))
    +      assert(catalog.getTempView("tbl4").isEmpty)
    +      assert(catalog.externalCatalog.listTables("db2").toSet == Set("tbl1", "tbl4"))
    +    }
       }
     
       test("alter table") {
    -    val externalCatalog = newBasicCatalog()
    -    val sessionCatalog = new SessionCatalog(externalCatalog)
    -    val tbl1 = externalCatalog.getTable("db2", "tbl1")
    -    sessionCatalog.alterTable(tbl1.copy(properties = Map("toh" -> "frem")))
    -    val newTbl1 = externalCatalog.getTable("db2", "tbl1")
    -    assert(!tbl1.properties.contains("toh"))
    -    assert(newTbl1.properties.size == tbl1.properties.size + 1)
    -    assert(newTbl1.properties.get("toh") == Some("frem"))
    -    // Alter table without explicitly specifying database
    -    sessionCatalog.setCurrentDatabase("db2")
    -    sessionCatalog.alterTable(tbl1.copy(identifier = TableIdentifier("tbl1")))
    -    val newestTbl1 = externalCatalog.getTable("db2", "tbl1")
    -    assert(newestTbl1 == tbl1)
    +    withBasicCatalog { catalog =>
    +      val tbl1 = catalog.externalCatalog.getTable("db2", "tbl1")
    +      catalog.alterTable(tbl1.copy(properties = Map("toh" -> "frem")))
    +      val newTbl1 = catalog.externalCatalog.getTable("db2", "tbl1")
    +      assert(!tbl1.properties.contains("toh"))
    +      assert(newTbl1.properties.size == tbl1.properties.size + 1)
    +      assert(newTbl1.properties.get("toh") == Some("frem"))
    +      // Alter table without explicitly specifying database
    +      catalog.setCurrentDatabase("db2")
    +      catalog.alterTable(tbl1.copy(identifier = TableIdentifier("tbl1")))
    +      val newestTbl1 = catalog.externalCatalog.getTable("db2", "tbl1")
    +      assert(newestTbl1.copy(properties = Map.empty) == tbl1.copy(properties = Map.empty))
    --- End diff --
    
    hive table properties will have transient_lastDdltime modified, so here we set it to empty
    and there is only this key exists in 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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106569958
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    here the `ignoreIfExists = true`, I think it is ok that the default database have already created.


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatal...

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

    https://github.com/apache/spark/pull/17287#discussion_r106570185
  
    --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala ---
    @@ -27,41 +27,67 @@ import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
     import org.apache.spark.sql.catalyst.plans.PlanTest
     import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
     
    +class InMemorySessionCatalogSuite extends SessionCatalogSuite {
    +  protected val utils = new CatalogTestUtils {
    +    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    +    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    +    override val defaultProvider: String = "parquet"
    +    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    +  }
    +}
    +
     /**
    - * Tests for [[SessionCatalog]] that assume that [[InMemoryCatalog]] is correctly implemented.
    + * Tests for [[SessionCatalog]]
      *
      * Note: many of the methods here are very similar to the ones in [[ExternalCatalogSuite]].
      * This is because [[SessionCatalog]] and [[ExternalCatalog]] share many similar method
      * signatures but do not extend a common parent. This is largely by design but
      * unfortunately leads to very similar test code in two places.
      */
    -class SessionCatalogSuite extends PlanTest {
    -  private val utils = new CatalogTestUtils {
    -    override val tableInputFormat: String = "com.fruit.eyephone.CameraInputFormat"
    -    override val tableOutputFormat: String = "com.fruit.eyephone.CameraOutputFormat"
    -    override val defaultProvider: String = "parquet"
    -    override def newEmptyCatalog(): ExternalCatalog = new InMemoryCatalog
    -  }
    +abstract class SessionCatalogSuite extends PlanTest {
    +  protected val utils: CatalogTestUtils
    +
    +  protected val isHiveExternalCatalog = false
     
       import utils._
     
    +  private def withBasicCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newBasicCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    +    try {
    +      f(catalog)
    +    } finally {
    +      catalog.reset()
    +    }
    +  }
    +
    +  private def withEmptyCatalog(f: SessionCatalog => Unit): Unit = {
    +    val catalog = new SessionCatalog(newEmptyCatalog())
    +    catalog.createDatabase(newDb("default"), ignoreIfExists = true)
    --- End diff --
    
    According to all the test cases in this Suite, it create the default database with `ignoreIfExists=true`, we have a default database existed here will not effect the test case


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74647/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test FAILed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74642/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

    https://github.com/apache/spark/pull/17287
  
    Test PASSed.
    Refer to this link for build results (access rights to CI server needed): 
    https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/74596/
    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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


---
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 #17287: [SPARK-19945][SQL]add test suite for SessionCatalog with...

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

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


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