spark
2ba1eba37121
[SPARK-12868][SQL] Allow adding jars from hdfs
Weiqing Yang
1 day ago
## What changes were proposed in this pull request? Spark 2.2 is going to be cut, it'll be great if SPARK-12868 can be resolved before that. There have been several PRs for this like [PR#16324](https://github.com/apache/spark/pull/16324) , but all of them are inactivity for a long time or have been closed. This PR added a SparkUrlStreamHandlerFactory, which relies on 'protocol' to choose the appropriate UrlStreamHandlerFactory like FsUrlStreamHandlerFactory to create URLStreamHandler. ## How was this patch tested? 1. Add a new unit test. 2. Check manually. Before: throw an exception with " failed unknown protocol: hdfs" <img width="914" alt="screen shot 2017-03-17 at 9 07 36 pm" src="https://cloud.githubusercontent.com/assets/8546874/24075277/5abe0a7c-0bd5-11e7-900e-ec3d3105da0b.png"> After: <img width="1148" alt="screen shot 2017-03-18 at 11 42 18 am" src="https://cloud.githubusercontent.com/assets/8546874/24075283/69382a60-0bd5-11e7-8d30-d9405c3aaaba.png"> Author: Weiqing Yang <yangweiqing001@gmail.com> Closes #17342 from weiqingy/SPARK-18910.

15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32
* limitations under the License. */ package org.apache.spark.sql.internal
import java.net.URL
import java.util.Locale import scala.reflect.ClassTag import scala.util.control.NonFatal import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.FsUrlStreamHandlerFactory
import org.apache.spark.{SparkConf, SparkContext, SparkException} import org.apache.spark.internal.Logging import org.apache.spark.sql.{SparkSession, SQLContext} import org.apache.spark.sql.catalyst.catalog._
...
152 153 154 155 156 157 158 159 160 161 162 154 155 156 157 158 159 160 161 162 163 164 165 166 167 168 169 170
} SparkSession.sqlListener.get() } }
object SharedState {
object SharedState extends Logging { try { URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory()) } catch { case e: Error => logWarning("URL.setURLStreamHandlerFactory failed to set FsUrlStreamHandlerFactory") }
private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog" private def externalCatalogClassName(conf: SparkConf): String = { conf.get(CATALOG_IMPLEMENTATION) match {

17 18 19 20 21 22 23 24 25 26 17 18 19 20 21 22 23 24 25 26 27
package org.apache.spark.sql import java.io.File import java.math.MathContext
import java.net.{MalformedURLException, URL}
import java.sql.Timestamp import java.util.concurrent.atomic.AtomicBoolean import org.apache.spark.{AccumulatorSuite, SparkException} import org.apache.spark.scheduler.{SparkListener, SparkListenerJobStart}
...
2604 2605 2606 2607 2608 2609 2605 2606 2607 2608 2609 2610 2611 2612 2613 2614 2615 2616 2617 2618 2619 2620 2621 2622
throw new AnalysisException("", None, None, plan = null) } catch { case ae: AnalysisException => assert(ae.plan == null && ae.getMessage == ae.getSimpleMessage) } }
test("SPARK-12868: Allow adding jars from hdfs ") { val jarFromHdfs = "hdfs://doesnotmatter/test.jar" val jarFromInvalidFs = "fffs://doesnotmatter/test.jar" // if 'hdfs' is not supported, MalformedURLException will be thrown new URL(jarFromHdfs) intercept[MalformedURLException] { new URL(jarFromInvalidFs) } }
}
About FluentSend Feedback