aboutsummaryrefslogtreecommitdiff
path: root/sql
diff options
context:
space:
mode:
authorLiwei Lin <lwlin7@gmail.com>2017-02-09 00:48:47 -0500
committergatorsmile <gatorsmile@gmail.com>2017-02-09 00:48:47 -0500
commit9d9d67c7957f7cbbdbe889bdbc073568b2bfbb16 (patch)
tree1f919b0c976d20e0493923b4aa1869f8125fb51a /sql
parent50a991264c16e4c4126e88668ef4fbd048c782b8 (diff)
downloadspark-9d9d67c7957f7cbbdbe889bdbc073568b2bfbb16.tar.gz
spark-9d9d67c7957f7cbbdbe889bdbc073568b2bfbb16.tar.bz2
spark-9d9d67c7957f7cbbdbe889bdbc073568b2bfbb16.zip
[SPARK-19265][SQL][FOLLOW-UP] Configurable `tableRelationCache` maximum size
## What changes were proposed in this pull request? SPARK-19265 had made table relation cache general; this follow-up aims to make `tableRelationCache`'s maximum size configurable. In order to do sanity-check, this patch also adds a `checkValue()` method to `TypedConfigBuilder`. ## How was this patch tested? new test case: `test("conf entry: checkValue()")` Author: Liwei Lin <lwlin7@gmail.com> Closes #16736 from lw-lin/conf.
Diffstat (limited to 'sql')
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala3
-rw-r--r--sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala6
-rw-r--r--sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala11
-rw-r--r--sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala16
4 files changed, 33 insertions, 3 deletions
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala
index 0b6fa56469..5f50ce1ba6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/CatalystConf.scala
@@ -34,6 +34,8 @@ trait CatalystConf {
def optimizerInSetConversionThreshold: Int
def maxCaseBranchesForCodegen: Int
+ def tableRelationCacheSize: Int
+
def runSQLonFile: Boolean
def warehousePath: String
@@ -69,6 +71,7 @@ case class SimpleCatalystConf(
optimizerMaxIterations: Int = 100,
optimizerInSetConversionThreshold: Int = 10,
maxCaseBranchesForCodegen: Int = 20,
+ tableRelationCacheSize: Int = 1000,
runSQLonFile: Boolean = true,
crossJoinEnabled: Boolean = false,
cboEnabled: Boolean = false,
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
index e9543f7987..dd0c5cb706 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
@@ -118,11 +118,11 @@ class SessionCatalog(
}
/**
- * A cache of qualified table name to table relation plan.
+ * A cache of qualified table names to table relation plans.
*/
val tableRelationCache: Cache[QualifiedTableName, LogicalPlan] = {
- // TODO: create a config instead of hardcode 1000 here.
- CacheBuilder.newBuilder().maximumSize(1000).build[QualifiedTableName, LogicalPlan]()
+ val cacheSize = conf.tableRelationCacheSize
+ CacheBuilder.newBuilder().maximumSize(cacheSize).build[QualifiedTableName, LogicalPlan]()
}
/**
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 8c77da1763..dc0f130406 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -786,6 +786,9 @@ private[sql] class SQLConf extends Serializable with CatalystConf with Logging {
def maxCaseBranchesForCodegen: Int = getConf(MAX_CASES_BRANCHES)
+ def tableRelationCacheSize: Int =
+ getConf(StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE)
+
def exchangeReuseEnabled: Boolean = getConf(EXCHANGE_REUSE_ENABLED)
def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE)
@@ -1034,6 +1037,14 @@ object StaticSQLConf {
.intConf
.createWithDefault(4000)
+ val FILESOURCE_TABLE_RELATION_CACHE_SIZE =
+ buildStaticConf("spark.sql.filesourceTableRelationCacheSize")
+ .internal()
+ .doc("The maximum size of the cache that maps qualified table names to table relation plans.")
+ .intConf
+ .checkValue(cacheSize => cacheSize >= 0, "The maximum size of the cache must not be negative")
+ .createWithDefault(1000)
+
// When enabling the debug, Spark SQL internal table properties are not filtered out; however,
// some related DDL commands (e.g., ANALYZE TABLE and CREATE TABLE LIKE) might not work properly.
val DEBUG_MODE = buildStaticConf("spark.sql.debug")
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala
index 6c12f0ff7d..0e3a5ca9d7 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/SQLConfEntrySuite.scala
@@ -171,4 +171,20 @@ class SQLConfEntrySuite extends SparkFunSuite {
buildConf(key).stringConf.createOptional
}
}
+
+ test("StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE") {
+ val confEntry = StaticSQLConf.FILESOURCE_TABLE_RELATION_CACHE_SIZE
+ assert(conf.getConf(confEntry) === 1000)
+
+ conf.setConf(confEntry, -1)
+ val e1 = intercept[IllegalArgumentException] {
+ conf.getConf(confEntry)
+ }
+ assert(e1.getMessage === "The maximum size of the cache must not be negative")
+
+ val e2 = intercept[IllegalArgumentException] {
+ conf.setConfString(confEntry.key, "-1")
+ }
+ assert(e2.getMessage === "The maximum size of the cache must not be negative")
+ }
}