Skip to content

Commit

Permalink
[SPARK-12489][CORE][SQL][MLIB] Fix minor issues found by FindBugs
Browse files Browse the repository at this point in the history
Include the following changes:

1. Close `java.sql.Statement`
2. Fix incorrect `asInstanceOf`.
3. Remove unnecessary `synchronized` and `ReentrantLock`.

Author: Shixiong Zhu <[email protected]>

Closes #10440 from zsxwing/findbugs.
  • Loading branch information
zsxwing committed Dec 28, 2015
1 parent fb572c6 commit 710b411
Show file tree
Hide file tree
Showing 7 changed files with 51 additions and 32 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -18,7 +18,6 @@
package org.apache.spark.scheduler.cluster.mesos

import java.io.File
import java.util.concurrent.locks.ReentrantLock
import java.util.{Collections, Date, List => JList}

import scala.collection.JavaConverters._
Expand Down Expand Up @@ -126,7 +125,7 @@ private[spark] class MesosClusterScheduler(
private val retainedDrivers = conf.getInt("spark.mesos.retainedDrivers", 200)
private val maxRetryWaitTime = conf.getInt("spark.mesos.cluster.retry.wait.max", 60) // 1 minute
private val schedulerState = engineFactory.createEngine("scheduler")
private val stateLock = new ReentrantLock()
private val stateLock = new Object()
private val finishedDrivers =
new mutable.ArrayBuffer[MesosClusterSubmissionState](retainedDrivers)
private var frameworkId: String = null
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -293,9 +293,7 @@ private class ServerConnection extends LauncherConnection {
protected void handle(Message msg) throws IOException {
try {
if (msg instanceof Hello) {
synchronized (timeout) {
timeout.cancel();
}
timeout.cancel();
timeout = null;
Hello hello = (Hello) msg;
ChildProcAppHandle handle = pending.remove(hello.secret);
Expand Down
2 changes: 1 addition & 1 deletion launcher/src/main/java/org/apache/spark/launcher/Main.java
Original file line number Diff line number Diff line change
Expand Up @@ -151,7 +151,7 @@ private static class MainClassOptionParser extends SparkSubmitOptionParser {

@Override
protected boolean handle(String opt, String value) {
if (opt == CLASS) {
if (CLASS.equals(opt)) {
className = value;
}
return false;
Expand Down
4 changes: 2 additions & 2 deletions mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala
Original file line number Diff line number Diff line change
Expand Up @@ -386,9 +386,9 @@ private[tree] object LearningNode {
var levelsToGo = indexToLevel(nodeIndex)
while (levelsToGo > 0) {
if ((nodeIndex & (1 << levelsToGo - 1)) == 0) {
tmpNode = tmpNode.leftChild.asInstanceOf[LearningNode]
tmpNode = tmpNode.leftChild.get
} else {
tmpNode = tmpNode.rightChild.asInstanceOf[LearningNode]
tmpNode = tmpNode.rightChild.get
}
levelsToGo -= 1
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -297,7 +297,12 @@ final class DataFrameWriter private[sql](df: DataFrame) {
if (!tableExists) {
val schema = JdbcUtils.schemaString(df, url)
val sql = s"CREATE TABLE $table ($schema)"
conn.createStatement.executeUpdate(sql)
val statement = conn.createStatement
try {
statement.executeUpdate(sql)
} finally {
statement.close()
}
}
} finally {
conn.close()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -122,30 +122,35 @@ private[sql] object JDBCRDD extends Logging {
val dialect = JdbcDialects.get(url)
val conn: Connection = getConnector(properties.getProperty("driver"), url, properties)()
try {
val rs = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0").executeQuery()
val statement = conn.prepareStatement(s"SELECT * FROM $table WHERE 1=0")
try {
val rsmd = rs.getMetaData
val ncols = rsmd.getColumnCount
val fields = new Array[StructField](ncols)
var i = 0
while (i < ncols) {
val columnName = rsmd.getColumnLabel(i + 1)
val dataType = rsmd.getColumnType(i + 1)
val typeName = rsmd.getColumnTypeName(i + 1)
val fieldSize = rsmd.getPrecision(i + 1)
val fieldScale = rsmd.getScale(i + 1)
val isSigned = rsmd.isSigned(i + 1)
val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls
val metadata = new MetadataBuilder().putString("name", columnName)
val columnType =
dialect.getCatalystType(dataType, typeName, fieldSize, metadata).getOrElse(
getCatalystType(dataType, fieldSize, fieldScale, isSigned))
fields(i) = StructField(columnName, columnType, nullable, metadata.build())
i = i + 1
val rs = statement.executeQuery()
try {
val rsmd = rs.getMetaData
val ncols = rsmd.getColumnCount
val fields = new Array[StructField](ncols)
var i = 0
while (i < ncols) {
val columnName = rsmd.getColumnLabel(i + 1)
val dataType = rsmd.getColumnType(i + 1)
val typeName = rsmd.getColumnTypeName(i + 1)
val fieldSize = rsmd.getPrecision(i + 1)
val fieldScale = rsmd.getScale(i + 1)
val isSigned = rsmd.isSigned(i + 1)
val nullable = rsmd.isNullable(i + 1) != ResultSetMetaData.columnNoNulls
val metadata = new MetadataBuilder().putString("name", columnName)
val columnType =
dialect.getCatalystType(dataType, typeName, fieldSize, metadata).getOrElse(
getCatalystType(dataType, fieldSize, fieldScale, isSigned))
fields(i) = StructField(columnName, columnType, nullable, metadata.build())
i = i + 1
}
return new StructType(fields)
} finally {
rs.close()
}
return new StructType(fields)
} finally {
rs.close()
statement.close()
}
} finally {
conn.close()
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -49,14 +49,26 @@ object JdbcUtils extends Logging {
// Somewhat hacky, but there isn't a good way to identify whether a table exists for all
// SQL database systems using JDBC meta data calls, considering "table" could also include
// the database name. Query used to find table exists can be overriden by the dialects.
Try(conn.prepareStatement(dialect.getTableExistsQuery(table)).executeQuery()).isSuccess
Try {
val statement = conn.prepareStatement(dialect.getTableExistsQuery(table))
try {
statement.executeQuery()
} finally {
statement.close()
}
}.isSuccess
}

/**
* Drops a table from the JDBC database.
*/
def dropTable(conn: Connection, table: String): Unit = {
conn.createStatement.executeUpdate(s"DROP TABLE $table")
val statement = conn.createStatement
try {
statement.executeUpdate(s"DROP TABLE $table")
} finally {
statement.close()
}
}

/**
Expand Down

0 comments on commit 710b411

Please sign in to comment.