rickyma commented on code in PR #727:
URL: https://github.com/apache/incubator-uniffle/pull/727#discussion_r1371239553


##########
client-spark/spark2/src/main/java/org/apache/spark/network/util/NettyUtils.java:
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.network.util;
+
+import java.lang.reflect.Field;
+import java.util.concurrent.ThreadFactory;
+
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.channel.Channel;
+import io.netty.channel.EventLoopGroup;
+import io.netty.channel.ServerChannel;
+import io.netty.channel.epoll.EpollEventLoopGroup;
+import io.netty.channel.epoll.EpollServerSocketChannel;
+import io.netty.channel.epoll.EpollSocketChannel;
+import io.netty.channel.nio.NioEventLoopGroup;
+import io.netty.channel.socket.nio.NioServerSocketChannel;
+import io.netty.channel.socket.nio.NioSocketChannel;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import io.netty.util.internal.PlatformDependent;
+import io.netty.util.internal.SystemPropertyUtil;
+
+/** copy from spark, In order to override the createPooledByteBufAllocator 
method,
+ * the property DEFAULT_TINY_CACHE_SIZE does not exist in netty>4.1.47. */
+public class NettyUtils {

Review Comment:
   We found a compatibility issue when using the Spark 2.1.0 version. The 
following error will be thrown in Spark driver's log:
   
   Exception in thread "main" java.lang.NoSuchMethodError: 
org.apache.spark.network.util.NettyUtils.createEventLoop(Lorg/apache/spark/network/util/IOMode;ILjava/lang/String;)Lio/netty/channel/EventLoopGroup;
        at 
org.apache.spark.network.client.TransportClientFactory.<init>(TransportClientFactory.java:104)
        at 
org.apache.spark.network.TransportContext.createClientFactory(TransportContext.java:89)
        at org.apache.spark.rpc.netty.NettyRpcEnv.<init>(NettyRpcEnv.scala:70)
        at 
org.apache.spark.rpc.netty.NettyRpcEnvFactory.create(NettyRpcEnv.scala:449)
        at org.apache.spark.rpc.RpcEnv$.create(RpcEnv.scala:56)
        at org.apache.spark.SparkEnv$.create(SparkEnv.scala:264)
        at org.apache.spark.SparkEnv$.createDriverEnv(SparkEnv.scala:193)
        at org.apache.spark.SparkContext.createSparkEnv(SparkContext.scala:271)
        at org.apache.spark.SparkContext.<init>(SparkContext.scala:474)
        at 
org.apache.spark.deploy.yarn.SQLApplicationMaster.<init>(SQLApplicationMaster.scala:96)
        at 
org.apache.spark.deploy.yarn.SQLApplicationMaster.<init>(SQLApplicationMaster.scala:53)
        at 
org.apache.spark.deploy.yarn.SQLApplicationMaster$$anonfun$main$1.apply$mcV$sp(SQLApplicationMaster.scala:544)
        at 
org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:61)
        at 
org.apache.spark.deploy.SparkHadoopUtil$$anon$1.run(SparkHadoopUtil.scala:60)
        at java.security.AccessController.doPrivileged(Native Method)
        at javax.security.auth.Subject.doAs(Subject.java:422)
        at 
org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:2337)
        at 
org.apache.spark.deploy.SparkHadoopUtil.runAsSparkUser(SparkHadoopUtil.scala:60)
        at 
org.apache.spark.deploy.yarn.SQLApplicationMaster$.main(SQLApplicationMaster.scala:543)
        at 
org.apache.spark.deploy.yarn.SQLApplicationMaster.main(SQLApplicationMaster.scala)
   
   This is because the return value of `createEventLoop` in `NettyUtils` within 
Uniffle is `org.apache.uniffle.io.netty.channel.EventLoopGroup` (which is 
shaded), while the return value of `createEventLoop` in `NettyUtils` within 
Spark is `io.netty.channel.EventLoopGroup`. When running a Spark application, 
the Driver loads `NettyUtils` from the rss-client's JAR, causing inconsistency 
in the method's return values and ultimately leading to a NoSuchMethodError 
exception.
   
   Any ideas? Thx a lot.
   @leixm @jerqi 



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to