|
17 | 17 |
|
18 | 18 | package org.apache.spark.sql.hive.thriftserver |
19 | 19 |
|
20 | | -import test.custom.listener.{DummyQueryExecutionListener, DummyStreamingQueryListener} |
21 | | - |
22 | 20 | import org.apache.spark.SparkFunSuite |
23 | 21 | import org.apache.spark.launcher.SparkLauncher |
24 | 22 | import org.apache.spark.sql.hive.HiveUtils.{HIVE_METASTORE_JARS, HIVE_METASTORE_VERSION} |
25 | 23 | import org.apache.spark.sql.hive.test.TestHiveContext |
| 24 | +import org.apache.spark.sql.hive.thriftserver.__root__.test.custom.listener.{DummyQueryExecutionListener, DummyStreamingQueryListener} |
26 | 25 | import org.apache.spark.sql.internal.StaticSQLConf.{QUERY_EXECUTION_LISTENERS, STREAMING_QUERY_LISTENERS, WAREHOUSE_PATH} |
27 | 26 |
|
28 | 27 | class SparkSQLEnvSuite extends SparkFunSuite { |
29 | 28 | test("SPARK-29604 external listeners should be initialized with Spark classloader") { |
30 | 29 | withSystemProperties( |
31 | | - // Intentionally place listeners to the out of spark package, because IsolatedClientLoader |
32 | | - // leverages Spark classloader for shared classess including spark package. |
33 | 30 | QUERY_EXECUTION_LISTENERS.key -> classOf[DummyQueryExecutionListener].getCanonicalName, |
34 | 31 | STREAMING_QUERY_LISTENERS.key -> classOf[DummyStreamingQueryListener].getCanonicalName, |
35 | 32 | WAREHOUSE_PATH.key -> TestHiveContext.makeWarehouseDir().toURI.getPath, |
@@ -68,3 +65,27 @@ class SparkSQLEnvSuite extends SparkFunSuite { |
68 | 65 | } |
69 | 66 | } |
70 | 67 | } |
| 68 | + |
| 69 | +/** |
| 70 | + * These classes in this package are intentionally placed to the outer package of spark, |
| 71 | + * because IsolatedClientLoader leverages Spark classloader for shared classess including |
| 72 | + * spark package, and the test should fail if Spark initializes these listeners with |
| 73 | + * IsolatedClientLoader. |
| 74 | + */ |
| 75 | +package __root__.test.custom.listener { |
| 76 | + |
| 77 | + import org.apache.spark.sql.execution.QueryExecution |
| 78 | + import org.apache.spark.sql.streaming.StreamingQueryListener |
| 79 | + import org.apache.spark.sql.util.QueryExecutionListener |
| 80 | + |
| 81 | + class DummyQueryExecutionListener extends QueryExecutionListener { |
| 82 | + override def onSuccess(funcName: String, qe: QueryExecution, durationNs: Long): Unit = {} |
| 83 | + override def onFailure(funcName: String, qe: QueryExecution, error: Throwable): Unit = {} |
| 84 | + } |
| 85 | + |
| 86 | + class DummyStreamingQueryListener extends StreamingQueryListener { |
| 87 | + override def onQueryStarted(event: StreamingQueryListener.QueryStartedEvent): Unit = {} |
| 88 | + override def onQueryProgress(event: StreamingQueryListener.QueryProgressEvent): Unit = {} |
| 89 | + override def onQueryTerminated(event: StreamingQueryListener.QueryTerminatedEvent): Unit = {} |
| 90 | + } |
| 91 | +} |
0 commit comments