Skip to content
Closed
Show file tree
Hide file tree
Changes from 1 commit
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Prev Previous commit
Next Next commit
Fix review comments
  • Loading branch information
weiqingy committed Apr 20, 2017
commit 47d36a10cdadf5b4c49db13a5851103b2edf24df
24 changes: 1 addition & 23 deletions core/src/main/scala/org/apache/spark/util/Utils.scala
Original file line number Diff line number Diff line change
Expand Up @@ -47,7 +47,7 @@ import com.google.common.io.{ByteStreams, Files => GFiles}
import com.google.common.net.InetAddresses
import org.apache.commons.lang3.SystemUtils
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{FileSystem, FileUtil, FsUrlStreamHandlerFactory, Path}
import org.apache.hadoop.fs.{FileSystem, FileUtil, Path}
import org.apache.hadoop.security.UserGroupInformation
import org.apache.log4j.PropertyConfigurator
import org.eclipse.jetty.util.MultiException
Expand Down Expand Up @@ -2780,25 +2780,3 @@ private[spark] class CircularBuffer(sizeInBytes: Int = 10240) extends java.io.Ou
new String(nonCircularBuffer, StandardCharsets.UTF_8)
}
}


/**
* Factory for URL stream handlers. It relies on 'protocol' to choose the appropriate
* UrlStreamHandlerFactory to create URLStreamHandler. Add new 'if' branches in
* 'createURLStreamHandler' like 'hdfsHandler' to support more protocols.
*/
private[spark] class SparkUrlStreamHandlerFactory extends URLStreamHandlerFactory {
private var hdfsHandler : URLStreamHandler = _

def createURLStreamHandler(protocol: String): URLStreamHandler = {
if (protocol.compareToIgnoreCase("hdfs") == 0) {
if (hdfsHandler == null) {
hdfsHandler = new FsUrlStreamHandlerFactory(SparkHadoopUtil.get.conf)
.createURLStreamHandler(protocol)
}
hdfsHandler
} else {
null
}
}
}
17 changes: 1 addition & 16 deletions core/src/test/scala/org/apache/spark/util/UtilsSuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -20,7 +20,7 @@ package org.apache.spark.util
import java.io.{ByteArrayInputStream, ByteArrayOutputStream, DataOutput, DataOutputStream, File,
FileOutputStream, PrintStream}
import java.lang.{Double => JDouble, Float => JFloat}
import java.net._
import java.net.{BindException, ServerSocket, URI}
import java.nio.{ByteBuffer, ByteOrder}
import java.nio.charset.StandardCharsets
import java.text.DecimalFormatSymbols
Expand Down Expand Up @@ -1021,19 +1021,4 @@ class UtilsSuite extends SparkFunSuite with ResetSystemProperties with Logging {
secretKeys.foreach { key => assert(redactedConf(key) === Utils.REDACTION_REPLACEMENT_TEXT) }
assert(redactedConf("spark.regular.property") === "not_a_secret")
}

test("SparkUrlStreamHandlerFactory") {
URL.setURLStreamHandlerFactory(new SparkUrlStreamHandlerFactory())

// if 'hdfs' is not supported, MalformedURLException will be thrown
new URL("hdfs://docs.oracle.com/test.jar")

var exceptionThrown: Boolean = false
try {
new URL("fffs://doesnotmatter")
} catch {
case e: MalformedURLException => exceptionThrown = true
}
assert(exceptionThrown === true)
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@ 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
Expand All @@ -32,7 +33,7 @@ import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.execution.CacheManager
import org.apache.spark.sql.execution.ui.{SQLListener, SQLTab}
import org.apache.spark.sql.internal.StaticSQLConf._
import org.apache.spark.util.{MutableURLClassLoader, SparkUrlStreamHandlerFactory, Utils}
import org.apache.spark.util.{MutableURLClassLoader, Utils}


/**
Expand Down Expand Up @@ -149,8 +150,7 @@ private[sql] class SharedState(val sparkContext: SparkContext) extends Logging {
}

object SharedState {

URL.setURLStreamHandlerFactory(new SparkUrlStreamHandlerFactory())
URL.setURLStreamHandlerFactory(new FsUrlStreamHandlerFactory())

private val HIVE_EXTERNAL_CATALOG_CLASS_NAME = "org.apache.spark.sql.hive.HiveExternalCatalog"

Expand Down
16 changes: 16 additions & 0 deletions sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,7 @@ 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

Expand Down Expand Up @@ -2606,4 +2607,19 @@ class SQLQuerySuite extends QueryTest with SharedSQLContext {
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)
var exceptionThrown: Boolean = false
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Replace this whole block with:

intercept[MalformedURLException] {
  new URL(jarFromInvalidFs)
}

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Thanks. PR has been updated.

try {
new URL(jarFromInvalidFs)
} catch {
case e: MalformedURLException => exceptionThrown = true
}
assert(exceptionThrown === true)
}
}