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

    https://github.com/apache/spark/pull/13127#discussion_r63458151
  
    --- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/client/VersionsSuite.scala ---
    @@ -130,109 +124,343 @@ class VersionsSuite extends SparkFunSuite with 
Logging {
         test(s"$version: create client") {
           client = null
           System.gc() // Hack to avoid SEGV on some JVM versions.
    +      val hadoopConf = new Configuration();
    +      hadoopConf.set("test", "success")
           client =
             IsolatedClientLoader.forVersion(
               hiveMetastoreVersion = version,
               hadoopVersion = VersionInfo.getVersion,
               sparkConf = sparkConf,
    -          hadoopConf = new Configuration(),
    +          hadoopConf,
               config = buildConf(),
               ivyPath = ivyPath).createClient()
         }
     
    +    def table(database: String, tableName: String): CatalogTable = {
    +      CatalogTable(
    +        identifier = TableIdentifier(tableName, Some(database)),
    +        tableType = CatalogTableType.MANAGED,
    +        schema = Seq(CatalogColumn("key", "int")),
    +        storage = CatalogStorageFormat(
    +          locationUri = None,
    +          inputFormat = 
Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName),
    +          outputFormat = Some(
    +            
classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, 
_]].getName),
    +          serde = 
Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()),
    +          compressed = false,
    +          serdeProperties = Map.empty
    +        ))
    +    }
    +
    +    
///////////////////////////////////////////////////////////////////////////
    +    // Database related API
    +    
///////////////////////////////////////////////////////////////////////////
    +
    +    val tempDatabasePath = Utils.createTempDir().getCanonicalPath
    +
         test(s"$version: createDatabase") {
    -      val db = CatalogDatabase("default", "desc", "loc", Map())
    -      client.createDatabase(db, ignoreIfExists = true)
    +      val defaultDB = CatalogDatabase("default", "desc", "loc", Map())
    +      client.createDatabase(defaultDB, ignoreIfExists = true)
    +      val tempDB = CatalogDatabase(
    +        "temporary", description = "test create", tempDatabasePath, Map())
    +      client.createDatabase(tempDB, ignoreIfExists = true)
    +    }
    +
    +    test(s"$version: setCurrentDatabase") {
    +      client.setCurrentDatabase("default")
    +    }
    +
    +    test(s"$version: getDatabase") {
    +      // No exception should be thrown
    +      client.getDatabase("default")
    +    }
    +
    +    test(s"$version: getDatabaseOption") {
    +      assert(client.getDatabaseOption("default").isDefined)
    +      assert(client.getDatabaseOption("nonexist") == None)
         }
     
    +    test(s"$version: listDatabases") {
    +      assert(client.listDatabases("defau.*") == Seq("default"))
    +    }
    +
    +    test(s"$version: alterDatabase") {
    +      val database = client.getDatabase("temporary").copy(properties = 
Map("flag" -> "true"))
    +      client.alterDatabase(database)
    +      assert(client.getDatabase("temporary").properties.contains("flag"))
    +    }
    +
    +    test(s"$version: dropDatabase") {
    +      assert(client.getDatabaseOption("temporary").isDefined)
    +      client.dropDatabase("temporary", ignoreIfNotExists = false, cascade 
= true)
    +      assert(client.getDatabaseOption("temporary").isEmpty)
    +    }
    +
    +    
///////////////////////////////////////////////////////////////////////////
    +    // Table related API
    +    
///////////////////////////////////////////////////////////////////////////
    +
         test(s"$version: createTable") {
    -      val table =
    -        CatalogTable(
    -          identifier = TableIdentifier("src", Some("default")),
    -          tableType = CatalogTableType.MANAGED,
    -          schema = Seq(CatalogColumn("key", "int")),
    -          storage = CatalogStorageFormat(
    -            locationUri = None,
    -            inputFormat = 
Some(classOf[org.apache.hadoop.mapred.TextInputFormat].getName),
    -            outputFormat = Some(
    -              
classOf[org.apache.hadoop.hive.ql.io.HiveIgnoreKeyTextOutputFormat[_, 
_]].getName),
    -            serde = 
Some(classOf[org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe].getName()),
    -            compressed = false,
    -            serdeProperties = Map.empty
    -          ))
    -
    -      client.createTable(table, ignoreIfExists = false)
    +      client.createTable(table("default", tableName = "src"), 
ignoreIfExists = false)
    +      // Creates a temporary table
    +      client.createTable(table("default", "temporary"), ignoreIfExists = 
false)
    +    }
    +
    +    test(s"$version: loadTable") {
    +      client.loadTable(
    +        emptyDir,
    +        tableName = "src",
    +        replace = false,
    +        holdDDLTime = false)
         }
     
         test(s"$version: getTable") {
    +      // No exception should be thrown
           client.getTable("default", "src")
         }
     
    -    test(s"$version: listTables") {
    -      assert(client.listTables("default") === Seq("src"))
    +    test(s"$version: getTableOption") {
    +      assert(client.getTableOption("default", "src").isDefined)
         }
     
    -    test(s"$version: getDatabase") {
    -      client.getDatabase("default")
    +    test(s"$version: alterTable(table: CatalogTable)") {
    +      val newTable = client.getTable("default", "src").copy(properties = 
Map("changed" -> ""))
    +      client.alterTable(newTable)
    +      assert(client.getTable("default", 
"src").properties.contains("changed"))
         }
     
    -    test(s"$version: alterTable") {
    -      client.alterTable(client.getTable("default", "src"))
    +    test(s"$version: alterTable(tableName: String, table: CatalogTable)") {
    +      val newTable = client.getTable("default", "src").copy(properties = 
Map("changedAgain" -> ""))
    +      client.alterTable("src", newTable)
    +      assert(client.getTable("default", 
"src").properties.contains("changedAgain"))
         }
     
    -    test(s"$version: set command") {
    -      client.runSqlHive("SET spark.sql.test.key=1")
    +    test(s"$version: listTables(database)") {
    +      assert(client.listTables("default") === Seq("src", "temporary"))
    +    }
    +
    +    test(s"$version: listTables(database, pattern)") {
    +      assert(client.listTables("default", pattern = "src") === Seq("src"))
    +      assert(client.listTables("default", pattern = "nonexist") === 
Seq.empty[String])
    +    }
    +
    +    test(s"$version: dropTable") {
    +      client.dropTable("default", tableName = "temporary", 
ignoreIfNotExists = false)
    +      assert(client.listTables("default") === Seq("src"))
         }
     
    -    test(s"$version: create partitioned table DDL") {
    -      client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY 
(key INT)")
    -      client.runSqlHive("ALTER TABLE src_part ADD PARTITION (key = '1')")
    +    
///////////////////////////////////////////////////////////////////////////
    +    // Partition related API
    +    
///////////////////////////////////////////////////////////////////////////
    +
    +    val storageFormat = CatalogStorageFormat(
    +      locationUri = None,
    +      inputFormat = None,
    +      outputFormat = None,
    +      serde = None,
    +      compressed = false,
    +      serdeProperties = Map.empty)
    +
    +
    +    test(s"$version: sql create partitioned table") {
    +      client.runSqlHive("CREATE TABLE src_part (value INT) PARTITIONED BY 
(key1 INT, key2 INT)")
         }
     
    -    test(s"$version: getPartitions") {
    -      client.getPartitions(client.getTable("default", "src_part"))
    +    test(s"$version: createPartitions") {
    +      val partition1 = CatalogTablePartition(Map("key1" -> "1", "key2" -> 
"1"), storageFormat)
    +      val partition2 = CatalogTablePartition(Map("key1" -> "1", "key2" -> 
"2"), storageFormat)
    +      client.createPartitions(
    +        "default", "src_part", Seq(partition1, partition2), ignoreIfExists 
= true)
    +    }
    +
    +    test(s"$version: getPartitions(catalogTable)") {
    +      assert(2 == client.getPartitions(client.getTable("default", 
"src_part")).size)
         }
     
         test(s"$version: getPartitionsByFilter") {
    -      client.getPartitionsByFilter(client.getTable("default", "src_part"), 
Seq(EqualTo(
    -        AttributeReference("key", IntegerType, 
false)(NamedExpression.newExprId),
    +      // Only one partition [1, 1] for key2 == 1
    +      val result = client.getPartitionsByFilter(client.getTable("default", 
"src_part"), Seq(EqualTo(
    +        AttributeReference("key2", IntegerType, 
false)(NamedExpression.newExprId),
             Literal(1))))
    +
    +      // Hive 0.12 doesn't support getPartitionsByFilter, it ignores the 
filter condition.
    +      if (version != "12") {
    +        assert(result.size == 1)
    +      }
    +    }
    +
    +    test(s"$version: getPartition") {
    +      // No exception should be thrown
    +      client.getPartition("default", "src_part", Map("key1" -> "1", "key2" 
-> "2"))
    +    }
    +
    +    test(s"$version: getPartitionOption(db: String, table: String, spec: 
TablePartitionSpec)") {
    +      val partition = client.getPartitionOption(
    +        "default", "src_part", Map("key1" -> "1", "key2" -> "2"))
    +      assert(partition.isDefined)
    +    }
    +
    +    test(s"$version: getPartitionOption(table: CatalogTable, spec: 
TablePartitionSpec)") {
    +      val partition = client.getPartitionOption(
    +        client.getTable("default", "src_part"), Map("key1" -> "1", "key2" 
-> "2"))
    +      assert(partition.isDefined)
    +    }
    +
    +    test(s"$version: getPartitions(db: String, table: String)") {
    +      assert(2 == client.getPartitions("default", "src_part", None).size)
         }
     
         test(s"$version: loadPartition") {
    +      val partSpec = new java.util.LinkedHashMap[String, String]
    +      partSpec.put("key1", "1")
    +      partSpec.put("key2", "2")
    +
           client.loadPartition(
             emptyDir,
             "default.src_part",
             partSpec,
    -        false,
    -        false,
    -        false,
    -        false)
    -    }
    -
    -    test(s"$version: loadTable") {
    -      client.loadTable(
    -        emptyDir,
    -        "src",
    -        false,
    -        false)
    +        replace = false,
    +        holdDDLTime = false,
    +        inheritTableSpecs = false,
    +        isSkewedStoreAsSubdir = false)
         }
     
         test(s"$version: loadDynamicPartitions") {
    +      val partSpec = new java.util.LinkedHashMap[String, String]
    +      partSpec.put("key1", "1")
    +      partSpec.put("key2", "") // Dynamic partition
    +
           client.loadDynamicPartitions(
             emptyDir,
             "default.src_part",
             partSpec,
    -        false,
    -        1,
    +        replace = false,
    +        numDP = 1,
             false,
             false)
         }
     
    -    test(s"$version: create index and reset") {
    +    test(s"$version: renamePartitions") {
    +      val oldSpec = Map("key1" -> "1", "key2" -> "1")
    +      val newSpec = Map("key1" -> "1", "key2" -> "3")
    +      client.renamePartitions("default", "src_part", Seq(oldSpec), 
Seq(newSpec))
    +
    +      // Checks the existence of the new partition (key1 = 1, key2 = 3)
    +      assert(client.getPartitionOption("default", "src_part", 
newSpec).isDefined)
    +    }
    +
    +    test(s"$version: alterPartitions") {
    +      val spec = Map("key1" -> "1", "key2" -> "2")
    +      val storage = storageFormat.copy(compressed = true)
    +      val partition = CatalogTablePartition(spec, storage)
    +      client.alterPartitions("default", "src_part", Seq(partition))
    +    }
    +
    +    test(s"$version: dropPartitions") {
    +      val spec = Map("key1" -> "1", "key2" -> "3")
    +      client.dropPartitions("default", "src_part", Seq(spec), 
ignoreIfNotExists = true)
    +      assert(client.getPartitionOption("default", "src_part", 
spec).isEmpty)
    +    }
    +
    +    
///////////////////////////////////////////////////////////////////////////
    +    // Function related API
    +    
///////////////////////////////////////////////////////////////////////////
    +
    +    def function(name: String, className: String): CatalogFunction = {
    +      CatalogFunction(
    +        FunctionIdentifier(name, Some("default")), className, 
Seq.empty[FunctionResource])
    +    }
    +
    +    test(s"$version: createFunction") {
    +      val functionClass = "org.apache.spark.MyFunc1"
    +      client.createFunction("default", function("func1", functionClass))
    +    }
    +
    +    test(s"$version: functionExists") {
    +      assert(client.functionExists("default", "func1") == true)
    +    }
    +
    +    test(s"$version: renameFunction") {
    +      client.renameFunction("default", "func1", "func2")
    +      assert(client.functionExists("default", "func2") == true)
    +    }
    +
    +    test(s"$version: alterFunction") {
    +      val functionClass = "org.apache.spark.MyFunc2"
    +      client.alterFunction("default", function("func2", functionClass))
    +    }
    +
    +    test(s"$version: getFunction") {
    +      // No exception should be thrown
    +      val func = client.getFunction("default", "func2")
    +      assert(func.className == "org.apache.spark.MyFunc2")
    +    }
    +
    +    test(s"$version: getFunctionOption") {
    +      assert(client.getFunctionOption("default", "func2").isDefined)
    +    }
    +
    +    test(s"$version: listFunctions") {
    +      assert(client.listFunctions("default", "fun.*").size == 1)
    +    }
    +
    +    test(s"$version: dropFunction") {
    +      client.dropFunction("default", "func2")
    +      assert(client.listFunctions("default", "fun.*").size == 0)
    +    }
    +
    +    
///////////////////////////////////////////////////////////////////////////
    +    // SQL related API
    +    
///////////////////////////////////////////////////////////////////////////
    +
    +    test(s"$version: sql set command") {
    +      client.runSqlHive("SET spark.sql.test.key=1")
    +    }
    +
    +    test(s"$version: sql create index and reset") {
           client.runSqlHive("CREATE TABLE indexed_table (key INT)")
           client.runSqlHive("CREATE INDEX index_1 ON TABLE indexed_table(key) 
" +
             "as 'COMPACT' WITH DEFERRED REBUILD")
    +    }
    +
    +    
///////////////////////////////////////////////////////////////////////////
    +    // Miscellaneous API
    +    
///////////////////////////////////////////////////////////////////////////
    +
    +    test(s"$version: version") {
    +      client.version.fullVersion
    +    }
    +
    +    test(s"$version: getConf") {
    +      assert("success" === client.getConf("test", null))
    +    }
    +
    +    test(s"$version: setOut") {
    +      client.setOut(new PrintStream(new ByteArrayOutputStream()))
    +    }
    +
    +    test(s"$version: setInfo") {
    +      client.setInfo(new PrintStream(new ByteArrayOutputStream()))
    +    }
    +
    +    test(s"$version: setError") {
    +      client.setError(new PrintStream(new ByteArrayOutputStream()))
    +    }
    +
    +    test(s"$version: addJar") {
    +      client.addJar(".")
    --- End diff --
    
    Will this actually add anything?


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

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

Reply via email to