Skip to content

Commit ab5adb7

Browse files
committed
[SPARK-7145] [CORE] commons-lang (2.x) classes used instead of commons-lang3 (3.x); commons-io used without dependency
Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava Author: Sean Owen <[email protected]> Closes #5703 from srowen/SPARK-7145 and squashes the following commits: 21fbe03 [Sean Owen] Remove use of commons-lang in favor of commons-lang3 classes; remove commons-io use in favor of Guava
1 parent 5d45e1f commit ab5adb7

File tree

10 files changed

+35
-27
lines changed

10 files changed

+35
-27
lines changed

core/src/test/scala/org/apache/spark/FileServerSuite.scala

Lines changed: 3 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,7 @@ import java.net.URI
2222
import java.util.jar.{JarEntry, JarOutputStream}
2323
import javax.net.ssl.SSLException
2424

25-
import com.google.common.io.ByteStreams
26-
import org.apache.commons.io.{FileUtils, IOUtils}
25+
import com.google.common.io.{ByteStreams, Files}
2726
import org.apache.commons.lang3.RandomUtils
2827
import org.scalatest.FunSuite
2928

@@ -239,7 +238,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
239238
def fileTransferTest(server: HttpFileServer, sm: SecurityManager = null): Unit = {
240239
val randomContent = RandomUtils.nextBytes(100)
241240
val file = File.createTempFile("FileServerSuite", "sslTests", tmpDir)
242-
FileUtils.writeByteArrayToFile(file, randomContent)
241+
Files.write(randomContent, file)
243242
server.addFile(file)
244243

245244
val uri = new URI(server.serverUri + "/files/" + file.getName)
@@ -254,7 +253,7 @@ class FileServerSuite extends FunSuite with LocalSparkContext {
254253
Utils.setupSecureURLConnection(connection, sm)
255254
}
256255

257-
val buf = IOUtils.toByteArray(connection.getInputStream)
256+
val buf = ByteStreams.toByteArray(connection.getInputStream)
258257
assert(buf === randomContent)
259258
}
260259

core/src/test/scala/org/apache/spark/metrics/InputOutputMetricsSuite.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,7 +21,7 @@ import java.io.{File, FileWriter, PrintWriter}
2121

2222
import scala.collection.mutable.ArrayBuffer
2323

24-
import org.apache.commons.lang.math.RandomUtils
24+
import org.apache.commons.lang3.RandomUtils
2525
import org.apache.hadoop.conf.Configuration
2626
import org.apache.hadoop.fs.{FileSystem, Path}
2727
import org.apache.hadoop.io.{LongWritable, Text}
@@ -60,7 +60,7 @@ class InputOutputMetricsSuite extends FunSuite with SharedSparkContext
6060
tmpFile = new File(testTempDir, getClass.getSimpleName + ".txt")
6161
val pw = new PrintWriter(new FileWriter(tmpFile))
6262
for (x <- 1 to numRecords) {
63-
pw.println(RandomUtils.nextInt(numBuckets))
63+
pw.println(RandomUtils.nextInt(0, numBuckets))
6464
}
6565
pw.close()
6666

core/src/test/scala/org/apache/spark/network/netty/NettyBlockTransferSecuritySuite.scala

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,22 +17,24 @@
1717

1818
package org.apache.spark.network.netty
1919

20+
import java.io.InputStreamReader
2021
import java.nio._
22+
import java.nio.charset.Charset
2123
import java.util.concurrent.TimeUnit
2224

2325
import scala.concurrent.duration._
2426
import scala.concurrent.{Await, Promise}
2527
import scala.util.{Failure, Success, Try}
2628

27-
import org.apache.commons.io.IOUtils
29+
import com.google.common.io.CharStreams
2830
import org.apache.spark.network.buffer.{ManagedBuffer, NioManagedBuffer}
2931
import org.apache.spark.network.shuffle.BlockFetchingListener
3032
import org.apache.spark.network.{BlockDataManager, BlockTransferService}
3133
import org.apache.spark.storage.{BlockId, ShuffleBlockId}
3234
import org.apache.spark.{SecurityManager, SparkConf}
3335
import org.mockito.Mockito._
3436
import org.scalatest.mock.MockitoSugar
35-
import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite, ShouldMatchers}
37+
import org.scalatest.{FunSuite, ShouldMatchers}
3638

3739
class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with ShouldMatchers {
3840
test("security default off") {
@@ -113,7 +115,9 @@ class NettyBlockTransferSecuritySuite extends FunSuite with MockitoSugar with Sh
113115

114116
val result = fetchBlock(exec0, exec1, "1", blockId) match {
115117
case Success(buf) =>
116-
IOUtils.toString(buf.createInputStream()) should equal(blockString)
118+
val actualString = CharStreams.toString(
119+
new InputStreamReader(buf.createInputStream(), Charset.forName("UTF-8")))
120+
actualString should equal(blockString)
117121
buf.release()
118122
Success()
119123
case Failure(t) =>

external/flume-sink/pom.xml

Lines changed: 4 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -35,6 +35,10 @@
3535
<url>http://spark.apache.org/</url>
3636

3737
<dependencies>
38+
<dependency>
39+
<groupId>org.apache.commons</groupId>
40+
<artifactId>commons-lang3</artifactId>
41+
</dependency>
3842
<dependency>
3943
<groupId>org.apache.flume</groupId>
4044
<artifactId>flume-ng-sdk</artifactId>

external/flume-sink/src/main/scala/org/apache/spark/streaming/flume/sink/SparkAvroCallbackHandler.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,9 +21,9 @@ import java.util.concurrent.atomic.AtomicLong
2121

2222
import scala.collection.mutable
2323

24-
import org.apache.flume.Channel
25-
import org.apache.commons.lang.RandomStringUtils
2624
import com.google.common.util.concurrent.ThreadFactoryBuilder
25+
import org.apache.flume.Channel
26+
import org.apache.commons.lang3.RandomStringUtils
2727

2828
/**
2929
* Class that implements the SparkFlumeProtocol, that is used by the Avro Netty Server to process

sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala

Lines changed: 5 additions & 1 deletion
Original file line numberDiff line numberDiff line change
@@ -20,7 +20,8 @@ package org.apache.spark.sql.jdbc
2020
import java.sql.{Connection, DriverManager, ResultSet, ResultSetMetaData, SQLException}
2121
import java.util.Properties
2222

23-
import org.apache.commons.lang.StringEscapeUtils.escapeSql
23+
import org.apache.commons.lang3.StringUtils
24+
2425
import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
2526
import org.apache.spark.rdd.RDD
2627
import org.apache.spark.sql.catalyst.expressions.{Row, SpecificMutableRow}
@@ -239,6 +240,9 @@ private[sql] class JDBCRDD(
239240
case _ => value
240241
}
241242

243+
private def escapeSql(value: String): String =
244+
if (value == null) null else StringUtils.replace(value, "'", "''")
245+
242246
/**
243247
* Turns a single Filter into a String representing a SQL expression.
244248
* Returns null for an unhandled filter.

sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/AbstractSparkSQLDriver.scala

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.thriftserver
1919

2020
import scala.collection.JavaConversions._
2121

22-
import org.apache.commons.lang.exception.ExceptionUtils
22+
import org.apache.commons.lang3.exception.ExceptionUtils
2323
import org.apache.hadoop.hive.metastore.api.{FieldSchema, Schema}
2424
import org.apache.hadoop.hive.ql.Driver
2525
import org.apache.hadoop.hive.ql.processors.CommandProcessorResponse
@@ -61,7 +61,7 @@ private[hive] abstract class AbstractSparkSQLDriver(
6161
} catch {
6262
case cause: Throwable =>
6363
logError(s"Failed in [$command]", cause)
64-
new CommandProcessorResponse(1, ExceptionUtils.getFullStackTrace(cause), null)
64+
new CommandProcessorResponse(1, ExceptionUtils.getStackTrace(cause), null)
6565
}
6666
}
6767

sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLCLIDriver.scala

Lines changed: 3 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -24,18 +24,16 @@ import java.util.{ArrayList => JArrayList}
2424

2525
import jline.{ConsoleReader, History}
2626

27-
import org.apache.commons.lang.StringUtils
27+
import org.apache.commons.lang3.StringUtils
2828
import org.apache.commons.logging.LogFactory
2929
import org.apache.hadoop.conf.Configuration
3030
import org.apache.hadoop.hive.cli.{CliDriver, CliSessionState, OptionsProcessor}
31-
import org.apache.hadoop.hive.common.LogUtils.LogInitializationException
32-
import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils, LogUtils}
31+
import org.apache.hadoop.hive.common.{HiveInterruptCallback, HiveInterruptUtils}
3332
import org.apache.hadoop.hive.conf.HiveConf
3433
import org.apache.hadoop.hive.ql.Driver
3534
import org.apache.hadoop.hive.ql.exec.Utilities
36-
import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor, CommandProcessorFactory}
35+
import org.apache.hadoop.hive.ql.processors.{AddResourceProcessor, SetProcessor, CommandProcessor}
3736
import org.apache.hadoop.hive.ql.session.SessionState
38-
import org.apache.hadoop.hive.shims.ShimLoader
3937
import org.apache.thrift.transport.TSocket
4038

4139
import org.apache.spark.Logging

sql/hive/src/test/java/org/apache/spark/sql/hive/execution/UDFListString.java

Lines changed: 3 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -17,10 +17,10 @@
1717

1818
package org.apache.spark.sql.hive.execution;
1919

20-
import org.apache.hadoop.hive.ql.exec.UDF;
21-
2220
import java.util.List;
23-
import org.apache.commons.lang.StringUtils;
21+
22+
import org.apache.commons.lang3.StringUtils;
23+
import org.apache.hadoop.hive.ql.exec.UDF;
2424

2525
public class UDFListString extends UDF {
2626

sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala

Lines changed: 4 additions & 5 deletions
Original file line numberDiff line numberDiff line change
@@ -23,7 +23,6 @@ import scala.collection.mutable.ArrayBuffer
2323

2424
import org.scalatest.BeforeAndAfterEach
2525

26-
import org.apache.commons.io.FileUtils
2726
import org.apache.hadoop.fs.Path
2827
import org.apache.hadoop.hive.metastore.TableType
2928
import org.apache.hadoop.hive.ql.metadata.Table
@@ -174,7 +173,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
174173
sql("SELECT * FROM jsonTable"),
175174
Row("a", "b"))
176175

177-
FileUtils.deleteDirectory(tempDir)
176+
Utils.deleteRecursively(tempDir)
178177
sparkContext.parallelize(("a1", "b1", "c1") :: Nil).toDF()
179178
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
180179

@@ -190,7 +189,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
190189
checkAnswer(
191190
sql("SELECT * FROM jsonTable"),
192191
Row("a1", "b1", "c1"))
193-
FileUtils.deleteDirectory(tempDir)
192+
Utils.deleteRecursively(tempDir)
194193
}
195194

196195
test("drop, change, recreate") {
@@ -212,7 +211,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
212211
sql("SELECT * FROM jsonTable"),
213212
Row("a", "b"))
214213

215-
FileUtils.deleteDirectory(tempDir)
214+
Utils.deleteRecursively(tempDir)
216215
sparkContext.parallelize(("a", "b", "c") :: Nil).toDF()
217216
.toJSON.saveAsTextFile(tempDir.getCanonicalPath)
218217

@@ -231,7 +230,7 @@ class MetastoreDataSourcesSuite extends QueryTest with BeforeAndAfterEach {
231230
checkAnswer(
232231
sql("SELECT * FROM jsonTable"),
233232
Row("a", "b", "c"))
234-
FileUtils.deleteDirectory(tempDir)
233+
Utils.deleteRecursively(tempDir)
235234
}
236235

237236
test("invalidate cache and reload") {

0 commit comments

Comments
 (0)