LuciferYang commented on code in PR #50022:
URL: https://github.com/apache/spark/pull/50022#discussion_r1964017993


##########
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/HiveClientImpl.scala:
##########
@@ -1407,13 +1409,62 @@ private[hive] object HiveClientImpl extends Logging {
       case _ =>
         new HiveConf(conf, classOf[HiveConf])
     }
-    try {
+    val hive = try {
       Hive.getWithoutRegisterFns(hiveConf)
     } catch {
       // SPARK-37069: not all Hive versions have the above method (e.g., Hive 
2.3.9 has it but
-      // 2.3.8 don't), therefore here we fallback when encountering the 
exception.
+      // 2.3.8 doesn't), therefore here we fallback when encountering the 
exception.
       case _: NoSuchMethodError =>
         Hive.get(hiveConf)
     }
+    configureMaxThriftMessageSize(hiveConf, hive.getMSC)
+    hive
+  }
+
+  // SPARK-49489: a surgery for Hive 2.3.10 due to lack of HIVE-26633
+  private def configureMaxThriftMessageSize(
+      hiveConf: HiveConf, msClient: IMetaStoreClient): Unit = try {
+    msClient match {
+      // Hive uses Java Dynamic Proxy to enhance the MetaStoreClient to 
support synchronization
+      // and retrying, we should unwrap and access the real MetaStoreClient 
instance firstly
+      case proxy if JdkProxy.isProxyClass(proxy.getClass) =>
+        JdkProxy.getInvocationHandler(proxy) match {
+          case syncHandler if 
syncHandler.getClass.getName.endsWith("SynchronizedHandler") =>
+            val realMscField = SparkClassUtils.classForName(

Review Comment:
   Then, perhaps we can try to refactor a bit here,  maybe like
   
   ```scala
   JdkProxy.getInvocationHandler(proxy) match {
       case syncHandler if 
syncHandler.getClass.getName.endsWith("SynchronizedHandler") =>
         val realMsc = getFieldValue(syncHandler, 
"client").asInstanceOf[IMetaStoreClient]
         configureMaxThriftMessageSize(hiveConf, realMsc)
   
       case retryHandler: RetryingMetaStoreClient =>
         val realMsc = getFieldValue(retryHandler, 
"base").asInstanceOf[IMetaStoreClient]
         configureMaxThriftMessageSize(hiveConf, realMsc)
   
       case _ => // do nothing
     }
   
   private def getFieldValue(obj: AnyRef, fieldName: String): AnyRef = {
     val field = obj.getClass.getDeclaredField(fieldName)
     field.setAccessible(true)
     field.get(obj)
   }
   ```
   
   hmm... Perhaps the exception type caught in the catch block needs to be 
changed.
   
   



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


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

Reply via email to