This is an automated email from the ASF dual-hosted git repository.

yikaifei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/kyuubi.git


The following commit(s) were added to refs/heads/master by this push:
     new bdf867b19 [KYUUBI #5193] Make Spark hive connector plugin compilable 
on Scala 2.13
bdf867b19 is described below

commit bdf867b19a253927022ea782e0e365e86f39493c
Author: liangbowen <[email protected]>
AuthorDate: Wed Aug 23 13:58:17 2023 +0800

    [KYUUBI #5193] Make Spark hive connector plugin compilable on Scala 2.13
    
    ### _Why are the changes needed?_
    
    - to make Spark SQL hive connector plugin compilable on Scala 2.13 with 
Spark 3.3/3.4
    - rename class name `FilePartitionReader` which is copied from Spark to 
`SparkFilePartitionReader`to fix the class mismatch error
    ```
    [ERROR] [Error] 
/Users/bw/dev/incubator-kyuubi/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/HivePartitionReaderFactory.scala:83:
 type mismatch;
     found   : 
Iterator[org.apache.kyuubi.spark.connector.hive.read.HivePartitionedFileReader[org.apache.spark.sql.catalyst.InternalRow]]
     required: 
Iterator[org.apache.spark.sql.execution.datasources.v2.PartitionedFileReader[org.apache.spark.sql.catalyst.InternalRow]]
    
    ```
    
    ### _How was this patch tested?_
    - [ ] Add some test cases that check the changes thoroughly including 
negative and positive cases if possible
    
    - [ ] Add screenshots for manual tests if appropriate
    
    - [ ] [Run 
test](https://kyuubi.readthedocs.io/en/master/contributing/code/testing.html#running-tests)
 locally before make a pull request
    
    ### _Was this patch authored or co-authored using generative AI tooling?_
    
    No.
    
    Closes #5193 from bowenliang123/scala213-hivecon.
    
    Closes #5193
    
    d8c6bf5f0 [liangbowen] defer toMap
    b20ad4eb1 [liangbowen] adapt spark hive connector plugin to Scala 2.13
    
    Authored-by: liangbowen <[email protected]>
    Signed-off-by: yikaifei <[email protected]>
---
 .../scala/org/apache/kyuubi/spark/connector/hive/HiveTableCatalog.scala | 2 +-
 .../kyuubi/spark/connector/hive/read/HivePartitionReaderFactory.scala   | 2 +-
 .../read/{FilePartitionReader.scala => SparkFilePartitionReader.scala}  | 2 +-
 .../org/apache/spark/sql/hive/kyuubi/connector/HiveBridgeHelper.scala   | 2 +-
 4 files changed, 4 insertions(+), 4 deletions(-)

diff --git 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/HiveTableCatalog.scala
 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/HiveTableCatalog.scala
index 41976b264..75804eb63 100644
--- 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/HiveTableCatalog.scala
+++ 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/HiveTableCatalog.scala
@@ -284,7 +284,7 @@ class HiveTableCatalog(sparkSession: SparkSession)
   private def toOptions(properties: Map[String, String]): Map[String, String] 
= {
     properties.filterKeys(_.startsWith(TableCatalog.OPTION_PREFIX)).map {
       case (key, value) => key.drop(TableCatalog.OPTION_PREFIX.length) -> value
-    }
+    }.toMap
   }
 
   override def listNamespaces(): Array[Array[String]] =
diff --git 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/HivePartitionReaderFactory.scala
 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/HivePartitionReaderFactory.scala
index 2b8e2ffd8..0cbd4f6ef 100644
--- 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/HivePartitionReaderFactory.scala
+++ 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/HivePartitionReaderFactory.scala
@@ -80,7 +80,7 @@ case class HivePartitionReaderFactory(
             partitionSchema,
             file.partitionValues))
       }
-    new FilePartitionReader[InternalRow](iter)
+    new SparkFilePartitionReader[InternalRow](iter)
   }
 
   def buildReaderInternal(
diff --git 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/FilePartitionReader.scala
 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/SparkFilePartitionReader.scala
similarity index 98%
rename from 
extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/FilePartitionReader.scala
rename to 
extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/SparkFilePartitionReader.scala
index 13b6d4c20..f785694d1 100644
--- 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/FilePartitionReader.scala
+++ 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/kyuubi/spark/connector/hive/read/SparkFilePartitionReader.scala
@@ -31,7 +31,7 @@ import 
org.apache.kyuubi.spark.connector.hive.HiveConnectorUtils
 // scalastyle:off line.size.limit
 // copy from 
https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/v2/FilePartitionReader.scala
 // scalastyle:on line.size.limit
-class FilePartitionReader[T](readers: Iterator[HivePartitionedFileReader[T]])
+class SparkFilePartitionReader[T](readers: 
Iterator[HivePartitionedFileReader[T]])
   extends PartitionReader[T] with Logging {
   private var currentReader: HivePartitionedFileReader[T] = _
 
diff --git 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/spark/sql/hive/kyuubi/connector/HiveBridgeHelper.scala
 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/spark/sql/hive/kyuubi/connector/HiveBridgeHelper.scala
index 95def8656..8e3a9cd3d 100644
--- 
a/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/spark/sql/hive/kyuubi/connector/HiveBridgeHelper.scala
+++ 
b/extensions/spark/kyuubi-spark-connector-hive/src/main/scala/org/apache/spark/sql/hive/kyuubi/connector/HiveBridgeHelper.scala
@@ -80,7 +80,7 @@ object HiveBridgeHelper {
             s"Unsupported partition transform: $transform")
       }
 
-      (identityCols, bucketSpec)
+      (identityCols.toSeq, bucketSpec)
     }
   }
 

Reply via email to