diff --git a/R/pkg/DESCRIPTION b/R/pkg/DESCRIPTION
index 00dde64324ae7..879c1f80f2c5d 100644
--- a/R/pkg/DESCRIPTION
+++ b/R/pkg/DESCRIPTION
@@ -32,10 +32,10 @@ Collate:
'pairRDD.R'
'DataFrame.R'
'SQLContext.R'
- 'catalog.R'
'WindowSpec.R'
'backend.R'
'broadcast.R'
+ 'catalog.R'
'client.R'
'context.R'
'deserialize.R'
@@ -44,6 +44,7 @@ Collate:
'jvm.R'
'mllib_classification.R'
'mllib_clustering.R'
+ 'mllib_fpm.R'
'mllib_recommendation.R'
'mllib_regression.R'
'mllib_stat.R'
diff --git a/R/pkg/NAMESPACE b/R/pkg/NAMESPACE
index c02046c94bf4d..9b7e95ce30acb 100644
--- a/R/pkg/NAMESPACE
+++ b/R/pkg/NAMESPACE
@@ -66,7 +66,10 @@ exportMethods("glm",
"spark.randomForest",
"spark.gbt",
"spark.bisectingKmeans",
- "spark.svmLinear")
+ "spark.svmLinear",
+ "spark.fpGrowth",
+ "spark.freqItemsets",
+ "spark.associationRules")
# Job group lifecycle management methods
export("setJobGroup",
diff --git a/R/pkg/R/generics.R b/R/pkg/R/generics.R
index 80283e48ced7b..945676c7f10b3 100644
--- a/R/pkg/R/generics.R
+++ b/R/pkg/R/generics.R
@@ -1445,6 +1445,18 @@ setGeneric("spark.posterior", function(object, newData) { standardGeneric("spark
#' @export
setGeneric("spark.perplexity", function(object, data) { standardGeneric("spark.perplexity") })
+#' @rdname spark.fpGrowth
+#' @export
+setGeneric("spark.fpGrowth", function(data, ...) { standardGeneric("spark.fpGrowth") })
+
+#' @rdname spark.fpGrowth
+#' @export
+setGeneric("spark.freqItemsets", function(object) { standardGeneric("spark.freqItemsets") })
+
+#' @rdname spark.fpGrowth
+#' @export
+setGeneric("spark.associationRules", function(object) { standardGeneric("spark.associationRules") })
+
#' @param object a fitted ML model object.
#' @param path the directory where the model is saved.
#' @param ... additional argument(s) passed to the method.
diff --git a/R/pkg/R/mllib_fpm.R b/R/pkg/R/mllib_fpm.R
new file mode 100644
index 0000000000000..96251b2c7c195
--- /dev/null
+++ b/R/pkg/R/mllib_fpm.R
@@ -0,0 +1,158 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+# mllib_fpm.R: Provides methods for MLlib frequent pattern mining algorithms integration
+
+#' S4 class that represents a FPGrowthModel
+#'
+#' @param jobj a Java object reference to the backing Scala FPGrowthModel
+#' @export
+#' @note FPGrowthModel since 2.2.0
+setClass("FPGrowthModel", slots = list(jobj = "jobj"))
+
+#' FP-growth
+#'
+#' A parallel FP-growth algorithm to mine frequent itemsets.
+#' For more details, see
+#' \href{https://spark.apache.org/docs/latest/mllib-frequent-pattern-mining.html#fp-growth}{
+#' FP-growth}.
+#'
+#' @param data A SparkDataFrame for training.
+#' @param minSupport Minimal support level.
+#' @param minConfidence Minimal confidence level.
+#' @param itemsCol Features column name.
+#' @param numPartitions Number of partitions used for fitting.
+#' @param ... additional argument(s) passed to the method.
+#' @return \code{spark.fpGrowth} returns a fitted FPGrowth model.
+#' @rdname spark.fpGrowth
+#' @name spark.fpGrowth
+#' @aliases spark.fpGrowth,SparkDataFrame-method
+#' @export
+#' @examples
+#' \dontrun{
+#' raw_data <- read.df(
+#' "data/mllib/sample_fpgrowth.txt",
+#' source = "csv",
+#' schema = structType(structField("raw_items", "string")))
+#'
+#' data <- selectExpr(raw_data, "split(raw_items, ' ') as items")
+#' model <- spark.fpGrowth(data)
+#'
+#' # Show frequent itemsets
+#' frequent_itemsets <- spark.freqItemsets(model)
+#' showDF(frequent_itemsets)
+#'
+#' # Show association rules
+#' association_rules <- spark.associationRules(model)
+#' showDF(association_rules)
+#'
+#' # Predict on new data
+#' new_itemsets <- data.frame(items = c("t", "t,s"))
+#' new_data <- selectExpr(createDataFrame(new_itemsets), "split(items, ',') as items")
+#' predict(model, new_data)
+#'
+#' # Save and load model
+#' path <- "/path/to/model"
+#' write.ml(model, path)
+#' read.ml(path)
+#'
+#' # Optional arguments
+#' baskets_data <- selectExpr(createDataFrame(itemsets), "split(items, ',') as baskets")
+#' another_model <- spark.fpGrowth(data, minSupport = 0.1, minConfidence = 0.5,
+#' itemsCol = "baskets", numPartitions = 10)
+#' }
+#' @note spark.fpGrowth since 2.2.0
+setMethod("spark.fpGrowth", signature(data = "SparkDataFrame"),
+ function(data, minSupport = 0.3, minConfidence = 0.8,
+ itemsCol = "items", numPartitions = NULL) {
+ if (!is.numeric(minSupport) || minSupport < 0 || minSupport > 1) {
+ stop("minSupport should be a number [0, 1].")
+ }
+ if (!is.numeric(minConfidence) || minConfidence < 0 || minConfidence > 1) {
+ stop("minConfidence should be a number [0, 1].")
+ }
+ if (!is.null(numPartitions)) {
+ numPartitions <- as.integer(numPartitions)
+ stopifnot(numPartitions > 0)
+ }
+
+ jobj <- callJStatic("org.apache.spark.ml.r.FPGrowthWrapper", "fit",
+ data@sdf, as.numeric(minSupport), as.numeric(minConfidence),
+ itemsCol, numPartitions)
+ new("FPGrowthModel", jobj = jobj)
+ })
+
+# Get frequent itemsets.
+
+#' @param object a fitted FPGrowth model.
+#' @return A \code{SparkDataFrame} with frequent itemsets.
+#' The \code{SparkDataFrame} contains two columns:
+#' \code{items} (an array of the same type as the input column)
+#' and \code{freq} (frequency of the itemset).
+#' @rdname spark.fpGrowth
+#' @aliases freqItemsets,FPGrowthModel-method
+#' @export
+#' @note spark.freqItemsets(FPGrowthModel) since 2.2.0
+setMethod("spark.freqItemsets", signature(object = "FPGrowthModel"),
+ function(object) {
+ dataFrame(callJMethod(object@jobj, "freqItemsets"))
+ })
+
+# Get association rules.
+
+#' @return A \code{SparkDataFrame} with association rules.
+#' The \code{SparkDataFrame} contains three columns:
+#' \code{antecedent} (an array of the same type as the input column),
+#' \code{consequent} (an array of the same type as the input column),
+#' and \code{condfidence} (confidence).
+#' @rdname spark.fpGrowth
+#' @aliases associationRules,FPGrowthModel-method
+#' @export
+#' @note spark.associationRules(FPGrowthModel) since 2.2.0
+setMethod("spark.associationRules", signature(object = "FPGrowthModel"),
+ function(object) {
+ dataFrame(callJMethod(object@jobj, "associationRules"))
+ })
+
+# Makes predictions based on generated association rules
+
+#' @param newData a SparkDataFrame for testing.
+#' @return \code{predict} returns a SparkDataFrame containing predicted values.
+#' @rdname spark.fpGrowth
+#' @aliases predict,FPGrowthModel-method
+#' @export
+#' @note predict(FPGrowthModel) since 2.2.0
+setMethod("predict", signature(object = "FPGrowthModel"),
+ function(object, newData) {
+ predict_internal(object, newData)
+ })
+
+# Saves the FPGrowth model to the output path.
+
+#' @param path the directory where the model is saved.
+#' @param overwrite logical value indicating whether to overwrite if the output path
+#' already exists. Default is FALSE which means throw exception
+#' if the output path exists.
+#' @rdname spark.fpGrowth
+#' @aliases write.ml,FPGrowthModel,character-method
+#' @export
+#' @seealso \link{read.ml}
+#' @note write.ml(FPGrowthModel, character) since 2.2.0
+setMethod("write.ml", signature(object = "FPGrowthModel", path = "character"),
+ function(object, path, overwrite = FALSE) {
+ write_internal(object, path, overwrite)
+ })
diff --git a/R/pkg/R/mllib_utils.R b/R/pkg/R/mllib_utils.R
index 04a0a6f944412..5dfef8625061b 100644
--- a/R/pkg/R/mllib_utils.R
+++ b/R/pkg/R/mllib_utils.R
@@ -118,6 +118,8 @@ read.ml <- function(path) {
new("BisectingKMeansModel", jobj = jobj)
} else if (isInstanceOf(jobj, "org.apache.spark.ml.r.LinearSVCWrapper")) {
new("LinearSVCModel", jobj = jobj)
+ } else if (isInstanceOf(jobj, "org.apache.spark.ml.r.FPGrowthWrapper")) {
+ new("FPGrowthModel", jobj = jobj)
} else {
stop("Unsupported model: ", jobj)
}
diff --git a/R/pkg/inst/tests/testthat/test_mllib_fpm.R b/R/pkg/inst/tests/testthat/test_mllib_fpm.R
new file mode 100644
index 0000000000000..c38f1133897dd
--- /dev/null
+++ b/R/pkg/inst/tests/testthat/test_mllib_fpm.R
@@ -0,0 +1,83 @@
+#
+# Licensed to the Apache Software Foundation (ASF) under one or more
+# contributor license agreements. See the NOTICE file distributed with
+# this work for additional information regarding copyright ownership.
+# The ASF licenses this file to You under the Apache License, Version 2.0
+# (the "License"); you may not use this file except in compliance with
+# the License. You may obtain a copy of the License at
+#
+# http://www.apache.org/licenses/LICENSE-2.0
+#
+# Unless required by applicable law or agreed to in writing, software
+# distributed under the License is distributed on an "AS IS" BASIS,
+# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+# See the License for the specific language governing permissions and
+# limitations under the License.
+#
+
+library(testthat)
+
+context("MLlib frequent pattern mining")
+
+# Tests for MLlib frequent pattern mining algorithms in SparkR
+sparkSession <- sparkR.session(enableHiveSupport = FALSE)
+
+test_that("spark.fpGrowth", {
+ data <- selectExpr(createDataFrame(data.frame(items = c(
+ "1,2",
+ "1,2",
+ "1,2,3",
+ "1,3"
+ ))), "split(items, ',') as items")
+
+ model <- spark.fpGrowth(data, minSupport = 0.3, minConfidence = 0.8, numPartitions = 1)
+
+ itemsets <- collect(spark.freqItemsets(model))
+
+ expected_itemsets <- data.frame(
+ items = I(list(list("3"), list("3", "1"), list("2"), list("2", "1"), list("1"))),
+ freq = c(2, 2, 3, 3, 4)
+ )
+
+ expect_equivalent(expected_itemsets, itemsets)
+
+ expected_association_rules <- data.frame(
+ antecedent = I(list(list("2"), list("3"))),
+ consequent = I(list(list("1"), list("1"))),
+ confidence = c(1, 1)
+ )
+
+ expect_equivalent(expected_association_rules, collect(spark.associationRules(model)))
+
+ new_data <- selectExpr(createDataFrame(data.frame(items = c(
+ "1,2",
+ "1,3",
+ "2,3"
+ ))), "split(items, ',') as items")
+
+ expected_predictions <- data.frame(
+ items = I(list(list("1", "2"), list("1", "3"), list("2", "3"))),
+ prediction = I(list(list(), list(), list("1")))
+ )
+
+ expect_equivalent(expected_predictions, collect(predict(model, new_data)))
+
+ modelPath <- tempfile(pattern = "spark-fpm", fileext = ".tmp")
+ write.ml(model, modelPath, overwrite = TRUE)
+ loaded_model <- read.ml(modelPath)
+
+ expect_equivalent(
+ itemsets,
+ collect(spark.freqItemsets(loaded_model)))
+
+ unlink(modelPath)
+
+ model_without_numpartitions <- spark.fpGrowth(data, minSupport = 0.3, minConfidence = 0.8)
+ expect_equal(
+ count(spark.freqItemsets(model_without_numpartitions)),
+ count(spark.freqItemsets(model))
+ )
+
+})
+
+sparkR.session.stop()
diff --git a/docs/cluster-overview.md b/docs/cluster-overview.md
index 814e4406cf435..a2ad958959a50 100644
--- a/docs/cluster-overview.md
+++ b/docs/cluster-overview.md
@@ -52,7 +52,11 @@ The system currently supports three cluster managers:
* [Apache Mesos](running-on-mesos.html) -- a general cluster manager that can also run Hadoop MapReduce
and service applications.
* [Hadoop YARN](running-on-yarn.html) -- the resource manager in Hadoop 2.
-
+* [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark) -- In addition to the above,
+there is experimental support for Kubernetes. Kubernetes is an open-source platform
+for providing container-centric infrastructure. Kubernetes support is being actively
+developed in an [apache-spark-on-k8s](https://github.com/apache-spark-on-k8s/) Github organization.
+For documentation, refer to that project's README.
# Submitting Applications
diff --git a/docs/index.md b/docs/index.md
index 19a9d3bfc6017..ad4f24ff1a5d1 100644
--- a/docs/index.md
+++ b/docs/index.md
@@ -115,6 +115,7 @@ options for deployment:
* [Mesos](running-on-mesos.html): deploy a private cluster using
[Apache Mesos](http://mesos.apache.org)
* [YARN](running-on-yarn.html): deploy Spark on top of Hadoop NextGen (YARN)
+ * [Kubernetes (experimental)](https://github.com/apache-spark-on-k8s/spark): deploy Spark on top of Kubernetes
**Other Documents:**
diff --git a/docs/monitoring.md b/docs/monitoring.md
index 6cbc6660e816c..4d0617d253b80 100644
--- a/docs/monitoring.md
+++ b/docs/monitoring.md
@@ -289,7 +289,7 @@ can be identified by their `[attempt-id]`. In the API listed below, when running
/applications/[app-id]/jobs |
A list of all jobs for a given application.
-
?status=[complete|succeeded|failed] list only jobs in the specific state.
+
?status=[running|succeeded|failed|unknown] list only jobs in the specific state.
|
diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala b/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala
new file mode 100644
index 0000000000000..b8151d8d90702
--- /dev/null
+++ b/mllib/src/main/scala/org/apache/spark/ml/r/FPGrowthWrapper.scala
@@ -0,0 +1,86 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.r
+
+import org.apache.hadoop.fs.Path
+import org.json4s.JsonDSL._
+import org.json4s.jackson.JsonMethods._
+
+import org.apache.spark.ml.fpm.{FPGrowth, FPGrowthModel}
+import org.apache.spark.ml.util._
+import org.apache.spark.sql.{DataFrame, Dataset}
+
+private[r] class FPGrowthWrapper private (val fpGrowthModel: FPGrowthModel) extends MLWritable {
+ def freqItemsets: DataFrame = fpGrowthModel.freqItemsets
+ def associationRules: DataFrame = fpGrowthModel.associationRules
+
+ def transform(dataset: Dataset[_]): DataFrame = {
+ fpGrowthModel.transform(dataset)
+ }
+
+ override def write: MLWriter = new FPGrowthWrapper.FPGrowthWrapperWriter(this)
+}
+
+private[r] object FPGrowthWrapper extends MLReadable[FPGrowthWrapper] {
+
+ def fit(
+ data: DataFrame,
+ minSupport: Double,
+ minConfidence: Double,
+ itemsCol: String,
+ numPartitions: Integer): FPGrowthWrapper = {
+ val fpGrowth = new FPGrowth()
+ .setMinSupport(minSupport)
+ .setMinConfidence(minConfidence)
+ .setItemsCol(itemsCol)
+
+ if (numPartitions != null && numPartitions > 0) {
+ fpGrowth.setNumPartitions(numPartitions)
+ }
+
+ val fpGrowthModel = fpGrowth.fit(data)
+
+ new FPGrowthWrapper(fpGrowthModel)
+ }
+
+ override def read: MLReader[FPGrowthWrapper] = new FPGrowthWrapperReader
+
+ class FPGrowthWrapperReader extends MLReader[FPGrowthWrapper] {
+ override def load(path: String): FPGrowthWrapper = {
+ val modelPath = new Path(path, "model").toString
+ val fPGrowthModel = FPGrowthModel.load(modelPath)
+
+ new FPGrowthWrapper(fPGrowthModel)
+ }
+ }
+
+ class FPGrowthWrapperWriter(instance: FPGrowthWrapper) extends MLWriter {
+ override protected def saveImpl(path: String): Unit = {
+ val modelPath = new Path(path, "model").toString
+ val rMetadataPath = new Path(path, "rMetadata").toString
+
+ val rMetadataJson: String = compact(render(
+ "class" -> instance.getClass.getName
+ ))
+
+ sc.parallelize(Seq(rMetadataJson), 1).saveAsTextFile(rMetadataPath)
+
+ instance.fpGrowthModel.save(modelPath)
+ }
+ }
+}
diff --git a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala
index 358e522dfe1c8..b30ce12bc6cc8 100644
--- a/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala
+++ b/mllib/src/main/scala/org/apache/spark/ml/r/RWrappers.scala
@@ -68,6 +68,8 @@ private[r] object RWrappers extends MLReader[Object] {
BisectingKMeansWrapper.load(path)
case "org.apache.spark.ml.r.LinearSVCWrapper" =>
LinearSVCWrapper.load(path)
+ case "org.apache.spark.ml.r.FPGrowthWrapper" =>
+ FPGrowthWrapper.load(path)
case _ =>
throw new SparkException(s"SparkR read.ml does not support load $className")
}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
index f2b6324db619a..257dc83621e98 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/LocalityPreferredContainerPlacementStrategy.scala
@@ -23,7 +23,6 @@ import scala.collection.JavaConverters._
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.yarn.api.records.{ContainerId, Resource}
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
-import org.apache.hadoop.yarn.util.RackResolver
import org.apache.spark.SparkConf
import org.apache.spark.internal.config._
@@ -83,7 +82,8 @@ private[yarn] case class ContainerLocalityPreferences(nodes: Array[String], rack
private[yarn] class LocalityPreferredContainerPlacementStrategy(
val sparkConf: SparkConf,
val yarnConf: Configuration,
- val resource: Resource) {
+ val resource: Resource,
+ resolver: SparkRackResolver) {
/**
* Calculate each container's node locality and rack locality
@@ -139,7 +139,7 @@ private[yarn] class LocalityPreferredContainerPlacementStrategy(
// still be allocated with new container request.
val hosts = preferredLocalityRatio.filter(_._2 > 0).keys.toArray
val racks = hosts.map { h =>
- RackResolver.resolve(yarnConf, h).getNetworkLocation
+ resolver.resolve(yarnConf, h)
}.toSet
containerLocalityPreferences += ContainerLocalityPreferences(hosts, racks.toArray)
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
new file mode 100644
index 0000000000000..c711d088f2116
--- /dev/null
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/SparkRackResolver.scala
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License. You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.deploy.yarn
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.yarn.util.RackResolver
+import org.apache.log4j.{Level, Logger}
+
+/**
+ * Wrapper around YARN's [[RackResolver]]. This allows Spark tests to easily override the
+ * default behavior, since YARN's class self-initializes the first time it's called, and
+ * future calls all use the initial configuration.
+ */
+private[yarn] class SparkRackResolver {
+
+ // RackResolver logs an INFO message whenever it resolves a rack, which is way too often.
+ if (Logger.getLogger(classOf[RackResolver]).getLevel == null) {
+ Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN)
+ }
+
+ def resolve(conf: Configuration, hostName: String): String = {
+ RackResolver.resolve(conf, hostName).getNetworkLocation()
+ }
+
+}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
index 25556763da904..ed77a6e4a1c7c 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnAllocator.scala
@@ -30,7 +30,6 @@ import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.api.AMRMClient
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
import org.apache.hadoop.yarn.conf.YarnConfiguration
-import org.apache.hadoop.yarn.util.RackResolver
import org.apache.log4j.{Level, Logger}
import org.apache.spark.{SecurityManager, SparkConf, SparkException}
@@ -65,16 +64,12 @@ private[yarn] class YarnAllocator(
amClient: AMRMClient[ContainerRequest],
appAttemptId: ApplicationAttemptId,
securityMgr: SecurityManager,
- localResources: Map[String, LocalResource])
+ localResources: Map[String, LocalResource],
+ resolver: SparkRackResolver)
extends Logging {
import YarnAllocator._
- // RackResolver logs an INFO message whenever it resolves a rack, which is way too often.
- if (Logger.getLogger(classOf[RackResolver]).getLevel == null) {
- Logger.getLogger(classOf[RackResolver]).setLevel(Level.WARN)
- }
-
// Visible for testing.
val allocatedHostToContainersMap = new HashMap[String, collection.mutable.Set[ContainerId]]
val allocatedContainerToHostMap = new HashMap[ContainerId, String]
@@ -159,7 +154,7 @@ private[yarn] class YarnAllocator(
// A container placement strategy based on pending tasks' locality preference
private[yarn] val containerPlacementStrategy =
- new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource)
+ new LocalityPreferredContainerPlacementStrategy(sparkConf, conf, resource, resolver)
/**
* Use a different clock for YarnAllocator. This is mainly used for testing.
@@ -424,7 +419,7 @@ private[yarn] class YarnAllocator(
// Match remaining by rack
val remainingAfterRackMatches = new ArrayBuffer[Container]
for (allocatedContainer <- remainingAfterHostMatches) {
- val rack = RackResolver.resolve(conf, allocatedContainer.getNodeId.getHost).getNetworkLocation
+ val rack = resolver.resolve(conf, allocatedContainer.getNodeId.getHost)
matchContainerToRequest(allocatedContainer, rack, containersToUse,
remainingAfterRackMatches)
}
diff --git a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
index 53fb467f6408d..72f4d273ab53b 100644
--- a/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
+++ b/resource-managers/yarn/src/main/scala/org/apache/spark/deploy/yarn/YarnRMClient.scala
@@ -75,7 +75,7 @@ private[spark] class YarnRMClient extends Logging {
registered = true
}
new YarnAllocator(driverUrl, driverRef, conf, sparkConf, amClient, getAttemptId(), securityMgr,
- localResources)
+ localResources, new SparkRackResolver())
}
/**
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
index fb80ff9f31322..b7f25656e49ac 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/LocalityPlacementStrategySuite.scala
@@ -17,10 +17,9 @@
package org.apache.spark.deploy.yarn
+import scala.collection.JavaConverters._
import scala.collection.mutable.{HashMap, HashSet, Set}
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic
-import org.apache.hadoop.net.DNSToSwitchMapping
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.conf.YarnConfiguration
import org.mockito.Mockito._
@@ -51,9 +50,6 @@ class LocalityPlacementStrategySuite extends SparkFunSuite {
private def runTest(): Unit = {
val yarnConf = new YarnConfiguration()
- yarnConf.setClass(
- CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
- classOf[MockResolver], classOf[DNSToSwitchMapping])
// The numbers below have been chosen to balance being large enough to replicate the
// original issue while not taking too long to run when the issue is fixed. The main
@@ -62,7 +58,7 @@ class LocalityPlacementStrategySuite extends SparkFunSuite {
val resource = Resource.newInstance(8 * 1024, 4)
val strategy = new LocalityPreferredContainerPlacementStrategy(new SparkConf(),
- yarnConf, resource)
+ yarnConf, resource, new MockResolver())
val totalTasks = 32 * 1024
val totalContainers = totalTasks / 16
diff --git a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
index fcc0594cf6d80..97b0e8aca3330 100644
--- a/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
+++ b/resource-managers/yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnAllocatorSuite.scala
@@ -17,12 +17,9 @@
package org.apache.spark.deploy.yarn
-import java.util.{Arrays, List => JList}
-
import scala.collection.JavaConverters._
-import org.apache.hadoop.fs.CommonConfigurationKeysPublic
-import org.apache.hadoop.net.DNSToSwitchMapping
+import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.yarn.api.records._
import org.apache.hadoop.yarn.client.api.AMRMClient
import org.apache.hadoop.yarn.client.api.AMRMClient.ContainerRequest
@@ -38,24 +35,16 @@ import org.apache.spark.rpc.RpcEndpointRef
import org.apache.spark.scheduler.SplitInfo
import org.apache.spark.util.ManualClock
-class MockResolver extends DNSToSwitchMapping {
+class MockResolver extends SparkRackResolver {
- override def resolve(names: JList[String]): JList[String] = {
- if (names.size > 0 && names.get(0) == "host3") Arrays.asList("/rack2")
- else Arrays.asList("/rack1")
+ override def resolve(conf: Configuration, hostName: String): String = {
+ if (hostName == "host3") "/rack2" else "/rack1"
}
- override def reloadCachedMappings() {}
-
- def reloadCachedMappings(names: JList[String]) {}
}
class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfterEach {
val conf = new YarnConfiguration()
- conf.setClass(
- CommonConfigurationKeysPublic.NET_TOPOLOGY_NODE_SWITCH_MAPPING_IMPL_KEY,
- classOf[MockResolver], classOf[DNSToSwitchMapping])
-
val sparkConf = new SparkConf()
sparkConf.set("spark.driver.host", "localhost")
sparkConf.set("spark.driver.port", "4040")
@@ -111,7 +100,8 @@ class YarnAllocatorSuite extends SparkFunSuite with Matchers with BeforeAndAfter
rmClient,
appAttemptId,
new SecurityManager(sparkConf),
- Map())
+ Map(),
+ new MockResolver())
}
def createContainer(host: String): Container = {
diff --git a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
index c4a590ec6916b..52b5b347fa9c7 100644
--- a/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
+++ b/sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4
@@ -56,6 +56,10 @@ singleTableIdentifier
: tableIdentifier EOF
;
+singleFunctionIdentifier
+ : functionIdentifier EOF
+ ;
+
singleDataType
: dataType EOF
;
@@ -493,6 +497,10 @@ tableIdentifier
: (db=identifier '.')? table=identifier
;
+functionIdentifier
+ : (db=identifier '.')? function=identifier
+ ;
+
namedExpression
: expression (AS? (identifier | identifierList))?
;
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
index da37eb00dcd97..206ae2f0e5eb1 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
@@ -92,7 +92,7 @@ object ScalaReflection extends ScalaReflection {
* Array[T]. Special handling is performed for primitive types to map them back to their raw
* JVM form instead of the Scala Array that handles auto boxing.
*/
- private def arrayClassFor(tpe: `Type`): DataType = ScalaReflectionLock.synchronized {
+ private def arrayClassFor(tpe: `Type`): ObjectType = ScalaReflectionLock.synchronized {
val cls = tpe match {
case t if t <:< definitions.IntTpe => classOf[Array[Int]]
case t if t <:< definitions.LongTpe => classOf[Array[Long]]
@@ -178,15 +178,17 @@ object ScalaReflection extends ScalaReflection {
* is [a: int, b: long], then we will hit runtime error and say that we can't construct class
* `Data` with int and long, because we lost the information that `b` should be a string.
*
- * This method help us "remember" the required data type by adding a `UpCast`. Note that we
- * don't need to cast struct type because there must be `UnresolvedExtractValue` or
- * `GetStructField` wrapping it, thus we only need to handle leaf type.
+ * This method help us "remember" the required data type by adding a `UpCast`. Note that we
+ * only need to do this for leaf nodes.
*/
def upCastToExpectedType(
expr: Expression,
expected: DataType,
walkedTypePath: Seq[String]): Expression = expected match {
case _: StructType => expr
+ case _: ArrayType => expr
+ // TODO: ideally we should also skip MapType, but nested StructType inside MapType is rare and
+ // it's not trivial to support by-name resolution for StructType inside MapType.
case _ => UpCast(expr, expected, walkedTypePath)
}
@@ -265,42 +267,48 @@ object ScalaReflection extends ScalaReflection {
case t if t <:< localTypeOf[Array[_]] =>
val TypeRef(_, _, Seq(elementType)) = t
+ val Schema(_, elementNullable) = schemaFor(elementType)
+ val className = getClassNameFromType(elementType)
+ val newTypePath = s"""- array element class: "$className"""" +: walkedTypePath
- // TODO: add runtime null check for primitive array
- val primitiveMethod = elementType match {
- case t if t <:< definitions.IntTpe => Some("toIntArray")
- case t if t <:< definitions.LongTpe => Some("toLongArray")
- case t if t <:< definitions.DoubleTpe => Some("toDoubleArray")
- case t if t <:< definitions.FloatTpe => Some("toFloatArray")
- case t if t <:< definitions.ShortTpe => Some("toShortArray")
- case t if t <:< definitions.ByteTpe => Some("toByteArray")
- case t if t <:< definitions.BooleanTpe => Some("toBooleanArray")
- case _ => None
+ val mapFunction: Expression => Expression = p => {
+ val converter = deserializerFor(elementType, Some(p), newTypePath)
+ if (elementNullable) {
+ converter
+ } else {
+ AssertNotNull(converter, newTypePath)
+ }
}
- primitiveMethod.map { method =>
- Invoke(getPath, method, arrayClassFor(elementType))
- }.getOrElse {
- val className = getClassNameFromType(elementType)
- val newTypePath = s"""- array element class: "$className"""" +: walkedTypePath
- Invoke(
- MapObjects(
- p => deserializerFor(elementType, Some(p), newTypePath),
- getPath,
- schemaFor(elementType).dataType),
- "array",
- arrayClassFor(elementType))
+ val arrayData = UnresolvedMapObjects(mapFunction, getPath)
+ val arrayCls = arrayClassFor(elementType)
+
+ if (elementNullable) {
+ Invoke(arrayData, "array", arrayCls)
+ } else {
+ val primitiveMethod = elementType match {
+ case t if t <:< definitions.IntTpe => "toIntArray"
+ case t if t <:< definitions.LongTpe => "toLongArray"
+ case t if t <:< definitions.DoubleTpe => "toDoubleArray"
+ case t if t <:< definitions.FloatTpe => "toFloatArray"
+ case t if t <:< definitions.ShortTpe => "toShortArray"
+ case t if t <:< definitions.ByteTpe => "toByteArray"
+ case t if t <:< definitions.BooleanTpe => "toBooleanArray"
+ case other => throw new IllegalStateException("expect primitive array element type " +
+ "but got " + other)
+ }
+ Invoke(arrayData, primitiveMethod, arrayCls)
}
case t if t <:< localTypeOf[Seq[_]] =>
val TypeRef(_, _, Seq(elementType)) = t
- val Schema(dataType, nullable) = schemaFor(elementType)
+ val Schema(_, elementNullable) = schemaFor(elementType)
val className = getClassNameFromType(elementType)
val newTypePath = s"""- array element class: "$className"""" +: walkedTypePath
val mapFunction: Expression => Expression = p => {
val converter = deserializerFor(elementType, Some(p), newTypePath)
- if (nullable) {
+ if (elementNullable) {
converter
} else {
AssertNotNull(converter, newTypePath)
@@ -312,7 +320,7 @@ object ScalaReflection extends ScalaReflection {
case NoSymbol => classOf[Seq[_]]
case _ => mirror.runtimeClass(t.typeSymbol.asClass)
}
- MapObjects(mapFunction, getPath, dataType, Some(cls))
+ UnresolvedMapObjects(mapFunction, getPath, Some(cls))
case t if t <:< localTypeOf[Map[_, _]] =>
// TODO: add walked type path for map
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SimpleCatalystConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SimpleCatalystConf.scala
deleted file mode 100644
index 8498cf1c9be79..0000000000000
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/SimpleCatalystConf.scala
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.catalyst
-
-import java.util.TimeZone
-
-import org.apache.spark.sql.internal.SQLConf
-
-
-/**
- * A SQLConf that can be used for local testing. This class is only here to minimize the change
- * for ticket SPARK-19944 (moves SQLConf from sql/core to sql/catalyst). This class should
- * eventually be removed (test cases should just create SQLConf and set values appropriately).
- */
-case class SimpleCatalystConf(
- override val caseSensitiveAnalysis: Boolean,
- override val orderByOrdinal: Boolean = true,
- override val groupByOrdinal: Boolean = true,
- override val optimizerMaxIterations: Int = 100,
- override val optimizerInSetConversionThreshold: Int = 10,
- override val maxCaseBranchesForCodegen: Int = 20,
- override val tableRelationCacheSize: Int = 1000,
- override val runSQLonFile: Boolean = true,
- override val crossJoinEnabled: Boolean = false,
- override val cboEnabled: Boolean = false,
- override val joinReorderEnabled: Boolean = false,
- override val joinReorderDPThreshold: Int = 12,
- override val starSchemaDetection: Boolean = false,
- override val warehousePath: String = "/user/hive/warehouse",
- override val sessionLocalTimeZone: String = TimeZone.getDefault().getID,
- override val maxNestedViewDepth: Int = 100,
- override val constraintPropagationEnabled: Boolean = true)
- extends SQLConf {
-
- override def clone(): SimpleCatalystConf = this.copy()
-}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
index 1b3a53c6359e6..c698ca6a8347c 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
@@ -26,7 +26,7 @@ import org.apache.spark.sql.catalyst.catalog._
import org.apache.spark.sql.catalyst.encoders.OuterScopes
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
-import org.apache.spark.sql.catalyst.expressions.objects.NewInstance
+import org.apache.spark.sql.catalyst.expressions.objects.{MapObjects, NewInstance, UnresolvedMapObjects}
import org.apache.spark.sql.catalyst.expressions.SubExprUtils._
import org.apache.spark.sql.catalyst.optimizer.BooleanSimplification
import org.apache.spark.sql.catalyst.plans._
@@ -34,6 +34,7 @@ import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _}
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.trees.TreeNodeRef
import org.apache.spark.sql.catalyst.util.toPrettySQL
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
/**
@@ -42,13 +43,13 @@ import org.apache.spark.sql.types._
* to resolve attribute references.
*/
object SimpleAnalyzer extends Analyzer(
- new SessionCatalog(
- new InMemoryCatalog,
- EmptyFunctionRegistry,
- new SimpleCatalystConf(caseSensitiveAnalysis = true)) {
- override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean) {}
- },
- new SimpleCatalystConf(caseSensitiveAnalysis = true))
+ new SessionCatalog(
+ new InMemoryCatalog,
+ EmptyFunctionRegistry,
+ new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)) {
+ override def createDatabase(dbDefinition: CatalogDatabase, ignoreIfExists: Boolean) {}
+ },
+ new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true))
/**
* Provides a way to keep state during the analysis, this enables us to decouple the concerns
@@ -89,11 +90,11 @@ object AnalysisContext {
*/
class Analyzer(
catalog: SessionCatalog,
- conf: CatalystConf,
+ conf: SQLConf,
maxIterations: Int)
extends RuleExecutor[LogicalPlan] with CheckAnalysis {
- def this(catalog: SessionCatalog, conf: CatalystConf) = {
+ def this(catalog: SessionCatalog, conf: SQLConf) = {
this(catalog, conf, conf.optimizerMaxIterations)
}
@@ -2226,8 +2227,21 @@ class Analyzer(
validateTopLevelTupleFields(deserializer, inputs)
val resolved = resolveExpression(
deserializer, LocalRelation(inputs), throws = true)
- validateNestedTupleFields(resolved)
- resolved
+ val result = resolved transformDown {
+ case UnresolvedMapObjects(func, inputData, cls) if inputData.resolved =>
+ inputData.dataType match {
+ case ArrayType(et, _) =>
+ val expr = MapObjects(func, inputData, et, cls) transformUp {
+ case UnresolvedExtractValue(child, fieldName) if child.resolved =>
+ ExtractValue(child, fieldName, resolver)
+ }
+ expr
+ case other =>
+ throw new AnalysisException("need an array field but got " + other.simpleString)
+ }
+ }
+ validateNestedTupleFields(result)
+ result
}
}
@@ -2331,7 +2345,7 @@ class Analyzer(
}
/**
- * Replace [[TimeZoneAwareExpression]] without [[TimeZone]] by its copy with session local
+ * Replace [[TimeZoneAwareExpression]] without timezone id by its copy with session local
* time zone.
*/
object ResolveTimeZone extends Rule[LogicalPlan] {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala
index 920033a9a8480..f8004ca300ac7 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveHints.scala
@@ -17,10 +17,10 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
+import org.apache.spark.sql.internal.SQLConf
/**
@@ -43,7 +43,7 @@ object ResolveHints {
*
* This rule must happen before common table expressions.
*/
- class ResolveBroadcastHints(conf: CatalystConf) extends Rule[LogicalPlan] {
+ class ResolveBroadcastHints(conf: SQLConf) extends Rule[LogicalPlan] {
private val BROADCAST_HINT_NAMES = Set("BROADCAST", "BROADCASTJOIN", "MAPJOIN")
def resolver: Resolver = conf.resolver
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
index d5b3ea8c37c66..a991dd96e2828 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveInlineTables.scala
@@ -19,16 +19,17 @@ package org.apache.spark.sql.catalyst.analysis
import scala.util.control.NonFatal
-import org.apache.spark.sql.catalyst.{CatalystConf, InternalRow}
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.expressions.{Cast, TimeZoneAwareExpression}
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StructField, StructType}
/**
* An analyzer rule that replaces [[UnresolvedInlineTable]] with [[LocalRelation]].
*/
-case class ResolveInlineTables(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class ResolveInlineTables(conf: SQLConf) extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
case table: UnresolvedInlineTable if table.expressionsResolved =>
validateInputDimension(table)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala
index 6b3bb68538dd1..8841309939c24 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/ResolveTableValuedFunctions.scala
@@ -17,9 +17,7 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.spark.{SparkConf, SparkContext}
import org.apache.spark.sql.catalyst.expressions.Expression
-import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Range}
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.types.{DataType, IntegerType, LongType}
@@ -105,7 +103,7 @@ object ResolveTableValuedFunctions extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case u: UnresolvedTableValuedFunction if u.functionArgs.forall(_.resolved) =>
- builtinFunctions.get(u.functionName) match {
+ builtinFunctions.get(u.functionName.toLowerCase()) match {
case Some(tvf) =>
val resolved = tvf.flatMap { case (argList, resolver) =>
argList.implicitCast(u.functionArgs) match {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
index 38a3d3de1288e..256b18771052a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinals.scala
@@ -17,17 +17,17 @@
package org.apache.spark.sql.catalyst.analysis
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.{Expression, Literal, SortOrder}
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, LogicalPlan, Sort}
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.trees.CurrentOrigin.withOrigin
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.IntegerType
/**
* Replaces ordinal in 'order by' or 'group by' with UnresolvedOrdinal expression.
*/
-class SubstituteUnresolvedOrdinals(conf: CatalystConf) extends Rule[LogicalPlan] {
+class SubstituteUnresolvedOrdinals(conf: SQLConf) extends Rule[LogicalPlan] {
private def isIntLiteral(e: Expression) = e match {
case Literal(_, IntegerType) => true
case _ => false
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala
index a5640a6c967a1..3bd54c257d98d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/view.scala
@@ -18,10 +18,10 @@
package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, View}
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
/**
* This file defines analysis rules related to views.
@@ -47,7 +47,7 @@ import org.apache.spark.sql.catalyst.rules.Rule
* This should be only done after the batch of Resolution, because the view attributes are not
* completely resolved during the batch of Resolution.
*/
-case class AliasViewChild(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class AliasViewChild(conf: SQLConf) extends Rule[LogicalPlan] {
override def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
case v @ View(desc, output, child) if child.resolved && output != child.output =>
val resolver = conf.resolver
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 72ab075408899..6f8c6ee2f0f44 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
@@ -35,6 +35,7 @@ import org.apache.spark.sql.catalyst.expressions.{Expression, ExpressionInfo}
import org.apache.spark.sql.catalyst.parser.{CatalystSqlParser, ParserInterface}
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias, View}
import org.apache.spark.sql.catalyst.util.StringUtils
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StructField, StructType}
object SessionCatalog {
@@ -52,7 +53,7 @@ class SessionCatalog(
val externalCatalog: ExternalCatalog,
globalTempViewManager: GlobalTempViewManager,
functionRegistry: FunctionRegistry,
- conf: CatalystConf,
+ conf: SQLConf,
hadoopConf: Configuration,
parser: ParserInterface,
functionResourceLoader: FunctionResourceLoader) extends Logging {
@@ -63,7 +64,7 @@ class SessionCatalog(
def this(
externalCatalog: ExternalCatalog,
functionRegistry: FunctionRegistry,
- conf: CatalystConf) {
+ conf: SQLConf) {
this(
externalCatalog,
new GlobalTempViewManager("global_temp"),
@@ -79,7 +80,7 @@ class SessionCatalog(
this(
externalCatalog,
new SimpleFunctionRegistry,
- SimpleCatalystConf(caseSensitiveAnalysis = true))
+ new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true))
}
/** List of temporary tables, mapping from table name to their logical plan. */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
index 70ed44e025f51..dc2e40424fd5f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala
@@ -20,15 +20,18 @@ package org.apache.spark.sql.catalyst.catalog
import java.net.URI
import java.util.Date
+import scala.collection.mutable
+
import com.google.common.base.Objects
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.{CatalystConf, FunctionIdentifier, InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, InternalRow, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, Cast, Literal}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, DateTimeUtils}
import org.apache.spark.sql.catalyst.util.quoteIdentifier
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
@@ -57,20 +60,25 @@ case class CatalogStorageFormat(
properties: Map[String, String]) {
override def toString: String = {
- val serdePropsToString = CatalogUtils.maskCredentials(properties) match {
- case props if props.isEmpty => ""
- case props => "Properties: " + props.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")
- }
- val output =
- Seq(locationUri.map("Location: " + _).getOrElse(""),
- inputFormat.map("InputFormat: " + _).getOrElse(""),
- outputFormat.map("OutputFormat: " + _).getOrElse(""),
- if (compressed) "Compressed" else "",
- serde.map("Serde: " + _).getOrElse(""),
- serdePropsToString)
- output.filter(_.nonEmpty).mkString("Storage(", ", ", ")")
+ toLinkedHashMap.map { case ((key, value)) =>
+ if (value.isEmpty) key else s"$key: $value"
+ }.mkString("Storage(", ", ", ")")
}
+ def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+ val map = new mutable.LinkedHashMap[String, String]()
+ locationUri.foreach(l => map.put("Location", l.toString))
+ serde.foreach(map.put("Serde Library", _))
+ inputFormat.foreach(map.put("InputFormat", _))
+ outputFormat.foreach(map.put("OutputFormat", _))
+ if (compressed) map.put("Compressed", "")
+ CatalogUtils.maskCredentials(properties) match {
+ case props if props.isEmpty => // No-op
+ case props =>
+ map.put("Properties", props.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]"))
+ }
+ map
+ }
}
object CatalogStorageFormat {
@@ -91,15 +99,28 @@ case class CatalogTablePartition(
storage: CatalogStorageFormat,
parameters: Map[String, String] = Map.empty) {
- override def toString: String = {
+ def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+ val map = new mutable.LinkedHashMap[String, String]()
val specString = spec.map { case (k, v) => s"$k=$v" }.mkString(", ")
- val output =
- Seq(
- s"Partition Values: [$specString]",
- s"$storage",
- s"Partition Parameters:{${parameters.map(p => p._1 + "=" + p._2).mkString(", ")}}")
+ map.put("Partition Values", s"[$specString]")
+ map ++= storage.toLinkedHashMap
+ if (parameters.nonEmpty) {
+ map.put("Partition Parameters", s"{${parameters.map(p => p._1 + "=" + p._2).mkString(", ")}}")
+ }
+ map
+ }
- output.filter(_.nonEmpty).mkString("CatalogPartition(\n\t", "\n\t", ")")
+ override def toString: String = {
+ toLinkedHashMap.map { case ((key, value)) =>
+ if (value.isEmpty) key else s"$key: $value"
+ }.mkString("CatalogPartition(\n\t", "\n\t", ")")
+ }
+
+ /** Readable string representation for the CatalogTablePartition. */
+ def simpleString: String = {
+ toLinkedHashMap.map { case ((key, value)) =>
+ if (value.isEmpty) key else s"$key: $value"
+ }.mkString("", "\n", "")
}
/** Return the partition location, assuming it is specified. */
@@ -154,6 +175,14 @@ case class BucketSpec(
}
s"$numBuckets buckets, $bucketString$sortString"
}
+
+ def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+ mutable.LinkedHashMap[String, String](
+ "Num Buckets" -> numBuckets.toString,
+ "Bucket Columns" -> bucketColumnNames.map(quoteIdentifier).mkString("[", ", ", "]"),
+ "Sort Columns" -> sortColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
+ )
+ }
}
/**
@@ -261,40 +290,50 @@ case class CatalogTable(
locationUri, inputFormat, outputFormat, serde, compressed, properties))
}
- override def toString: String = {
+
+ def toLinkedHashMap: mutable.LinkedHashMap[String, String] = {
+ val map = new mutable.LinkedHashMap[String, String]()
val tableProperties = properties.map(p => p._1 + "=" + p._2).mkString("[", ", ", "]")
val partitionColumns = partitionColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
- val bucketStrings = bucketSpec match {
- case Some(BucketSpec(numBuckets, bucketColumnNames, sortColumnNames)) =>
- val bucketColumnsString = bucketColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
- val sortColumnsString = sortColumnNames.map(quoteIdentifier).mkString("[", ", ", "]")
- Seq(
- s"Num Buckets: $numBuckets",
- if (bucketColumnNames.nonEmpty) s"Bucket Columns: $bucketColumnsString" else "",
- if (sortColumnNames.nonEmpty) s"Sort Columns: $sortColumnsString" else ""
- )
-
- case _ => Nil
+
+ identifier.database.foreach(map.put("Database", _))
+ map.put("Table", identifier.table)
+ if (owner.nonEmpty) map.put("Owner", owner)
+ map.put("Created", new Date(createTime).toString)
+ map.put("Last Access", new Date(lastAccessTime).toString)
+ map.put("Type", tableType.name)
+ provider.foreach(map.put("Provider", _))
+ bucketSpec.foreach(map ++= _.toLinkedHashMap)
+ comment.foreach(map.put("Comment", _))
+ if (tableType == CatalogTableType.VIEW) {
+ viewText.foreach(map.put("View Text", _))
+ viewDefaultDatabase.foreach(map.put("View Default Database", _))
+ if (viewQueryColumnNames.nonEmpty) {
+ map.put("View Query Output Columns", viewQueryColumnNames.mkString("[", ", ", "]"))
+ }
}
- val output =
- Seq(s"Table: ${identifier.quotedString}",
- if (owner.nonEmpty) s"Owner: $owner" else "",
- s"Created: ${new Date(createTime).toString}",
- s"Last Access: ${new Date(lastAccessTime).toString}",
- s"Type: ${tableType.name}",
- if (schema.nonEmpty) s"Schema: ${schema.mkString("[", ", ", "]")}" else "",
- if (provider.isDefined) s"Provider: ${provider.get}" else "",
- if (partitionColumnNames.nonEmpty) s"Partition Columns: $partitionColumns" else ""
- ) ++ bucketStrings ++ Seq(
- viewText.map("View: " + _).getOrElse(""),
- comment.map("Comment: " + _).getOrElse(""),
- if (properties.nonEmpty) s"Properties: $tableProperties" else "",
- if (stats.isDefined) s"Statistics: ${stats.get.simpleString}" else "",
- s"$storage",
- if (tracksPartitionsInCatalog) "Partition Provider: Catalog" else "")
-
- output.filter(_.nonEmpty).mkString("CatalogTable(\n\t", "\n\t", ")")
+ if (properties.nonEmpty) map.put("Properties", tableProperties)
+ stats.foreach(s => map.put("Statistics", s.simpleString))
+ map ++= storage.toLinkedHashMap
+ if (tracksPartitionsInCatalog) map.put("Partition Provider", "Catalog")
+ if (partitionColumnNames.nonEmpty) map.put("Partition Columns", partitionColumns)
+ if (schema.nonEmpty) map.put("Schema", schema.treeString)
+
+ map
+ }
+
+ override def toString: String = {
+ toLinkedHashMap.map { case ((key, value)) =>
+ if (value.isEmpty) key else s"$key: $value"
+ }.mkString("CatalogTable(\n", "\n", ")")
+ }
+
+ /** Readable string representation for the CatalogTable. */
+ def simpleString: String = {
+ toLinkedHashMap.map { case ((key, value)) =>
+ if (value.isEmpty) key else s"$key: $value"
+ }.mkString("", "\n", "")
}
}
@@ -387,7 +426,7 @@ case class CatalogRelation(
/** Only compare table identifier. */
override lazy val cleanArgs: Seq[Any] = Seq(tableMeta.identifier)
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
// For data source tables, we will create a `LogicalRelation` and won't call this method, for
// hive serde tables, we will always generate a statistics.
// TODO: unify the table stats generation.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
index de1594d119e17..ef88cfb543ebb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypeExtractors.scala
@@ -68,7 +68,7 @@ object ExtractValue {
case StructType(_) =>
s"Field name should be String Literal, but it's $extraction"
case other =>
- s"Can't extract value from $child"
+ s"Can't extract value from $child: need struct type but got ${other.simpleString}"
}
throw new AnalysisException(errorMsg)
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
index bb584f7d087e8..00e2ac91e67ca 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/objects/objects.scala
@@ -448,6 +448,17 @@ object MapObjects {
}
}
+case class UnresolvedMapObjects(
+ function: Expression => Expression,
+ child: Expression,
+ customCollectionCls: Option[Class[_]] = None) extends UnaryExpression with Unevaluable {
+ override lazy val resolved = false
+
+ override def dataType: DataType = customCollectionCls.map(ObjectType.apply).getOrElse {
+ throw new UnsupportedOperationException("not resolved")
+ }
+}
+
/**
* Applies the given expression to every element of a collection of items, returning the result
* as an ArrayType or ObjectType. This is similar to a typical map operation, but where the lambda
@@ -581,17 +592,24 @@ case class MapObjects private(
// collection
val collObjectName = s"${cls.getName}$$.MODULE$$"
val getBuilderVar = s"$collObjectName.newBuilder()"
-
- (s"""${classOf[Builder[_, _]].getName} $builderValue = $getBuilderVar;
- $builderValue.sizeHint($dataLength);""",
+ (
+ s"""
+ ${classOf[Builder[_, _]].getName} $builderValue = $getBuilderVar;
+ $builderValue.sizeHint($dataLength);
+ """,
genValue => s"$builderValue.$$plus$$eq($genValue);",
- s"(${cls.getName}) $builderValue.result();")
+ s"(${cls.getName}) $builderValue.result();"
+ )
case None =>
// array
- (s"""$convertedType[] $convertedArray = null;
- $convertedArray = $arrayConstructor;""",
+ (
+ s"""
+ $convertedType[] $convertedArray = null;
+ $convertedArray = $arrayConstructor;
+ """,
genValue => s"$convertedArray[$loopIndex] = $genValue;",
- s"new ${classOf[GenericArrayData].getName}($convertedArray);")
+ s"new ${classOf[GenericArrayData].getName}($convertedArray);"
+ )
}
val code = s"""
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
index dbf479d215134..577112779eea4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
@@ -20,7 +20,6 @@ package org.apache.spark.sql.catalyst.optimizer
import scala.collection.mutable
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.expressions._
@@ -28,13 +27,14 @@ import org.apache.spark.sql.catalyst.expressions.aggregate._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
/**
* Abstract class all optimizers should inherit of, contains the standard batches (extending
* Optimizers can override this.
*/
-abstract class Optimizer(sessionCatalog: SessionCatalog, conf: CatalystConf)
+abstract class Optimizer(sessionCatalog: SessionCatalog, conf: SQLConf)
extends RuleExecutor[LogicalPlan] {
protected val fixedPoint = FixedPoint(conf.optimizerMaxIterations)
@@ -160,8 +160,8 @@ class SimpleTestOptimizer extends Optimizer(
new SessionCatalog(
new InMemoryCatalog,
EmptyFunctionRegistry,
- new SimpleCatalystConf(caseSensitiveAnalysis = true)),
- new SimpleCatalystConf(caseSensitiveAnalysis = true))
+ new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)),
+ new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true))
/**
* Remove redundant aliases from a query plan. A redundant alias is an alias that does not change
@@ -270,7 +270,7 @@ object RemoveRedundantProject extends Rule[LogicalPlan] {
/**
* Pushes down [[LocalLimit]] beneath UNION ALL and beneath the streamed inputs of outer joins.
*/
-case class LimitPushDown(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class LimitPushDown(conf: SQLConf) extends Rule[LogicalPlan] {
private def stripGlobalLimitIfPresent(plan: LogicalPlan): LogicalPlan = {
plan match {
@@ -617,7 +617,7 @@ object CollapseWindow extends Rule[LogicalPlan] {
* Note: While this optimization is applicable to all types of join, it primarily benefits Inner and
* LeftSemi joins.
*/
-case class InferFiltersFromConstraints(conf: CatalystConf)
+case class InferFiltersFromConstraints(conf: SQLConf)
extends Rule[LogicalPlan] with PredicateHelper {
def apply(plan: LogicalPlan): LogicalPlan = if (conf.constraintPropagationEnabled) {
inferFilters(plan)
@@ -715,7 +715,7 @@ object EliminateSorts extends Rule[LogicalPlan] {
* 2) by substituting a dummy empty relation when the filter will always evaluate to `false`.
* 3) by eliminating the always-true conditions given the constraints on the child's output.
*/
-case class PruneFilters(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
+case class PruneFilters(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHelper {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
// If the filter condition always evaluate to true, remove the filter.
case Filter(Literal(true, BooleanType), child) => child
@@ -1057,7 +1057,7 @@ object CombineLimits extends Rule[LogicalPlan] {
* the join between R and S is not a cartesian product and therefore should be allowed.
* The predicate R.r = S.s is not recognized as a join condition until the ReorderJoin rule.
*/
-case class CheckCartesianProducts(conf: CatalystConf)
+case class CheckCartesianProducts(conf: SQLConf)
extends Rule[LogicalPlan] with PredicateHelper {
/**
* Check if a join is a cartesian product. Returns true if
@@ -1092,7 +1092,7 @@ case class CheckCartesianProducts(conf: CatalystConf)
* This uses the same rules for increasing the precision and scale of the output as
* [[org.apache.spark.sql.catalyst.analysis.DecimalPrecision]].
*/
-case class DecimalAggregates(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class DecimalAggregates(conf: SQLConf) extends Rule[LogicalPlan] {
import Decimal.MAX_LONG_DIGITS
/** Maximum number of decimal digits representable precisely in a Double */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
index 33039127f16ce..8445ee06bd89b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/expressions.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.optimizer
import scala.collection.immutable.HashSet
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate._
@@ -27,6 +26,7 @@ import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLite
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
/*
@@ -115,7 +115,7 @@ object ReorderAssociativeOperator extends Rule[LogicalPlan] {
* 2. Replaces [[In (value, seq[Literal])]] with optimized version
* [[InSet (value, HashSet[Literal])]] which is much faster.
*/
-case class OptimizeIn(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class OptimizeIn(conf: SQLConf) extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
case q: LogicalPlan => q transformExpressionsDown {
case expr @ In(v, list) if expr.inSetConvertible =>
@@ -346,7 +346,7 @@ object LikeSimplification extends Rule[LogicalPlan] {
* equivalent [[Literal]] values. This rule is more specific with
* Null value propagation from bottom to top of the expression tree.
*/
-case class NullPropagation(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class NullPropagation(conf: SQLConf) extends Rule[LogicalPlan] {
private def isNullLiteral(e: Expression): Boolean = e match {
case Literal(null, _) => true
case _ => false
@@ -482,7 +482,7 @@ object FoldablePropagation extends Rule[LogicalPlan] {
/**
* Optimizes expressions by replacing according to CodeGen configuration.
*/
-case class OptimizeCodegen(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class OptimizeCodegen(conf: SQLConf) extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions {
case e: CaseWhen if canCodegen(e) => e.toCodegen()
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
index 5f7316566b3ba..250dd07a16eb4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.optimizer
import scala.annotation.tailrec
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning.{ExtractFiltersAndInnerJoins, PhysicalOperation}
import org.apache.spark.sql.catalyst.plans._
@@ -440,7 +439,7 @@ case class ReorderJoin(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHe
*
* This rule should be executed before pushing down the Filter
*/
-case class EliminateOuterJoin(conf: CatalystConf) extends Rule[LogicalPlan] with PredicateHelper {
+case class EliminateOuterJoin(conf: SQLConf) extends Rule[LogicalPlan] with PredicateHelper {
/**
* Returns whether the expression returns null or false when all inputs are nulls.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
index 4af56afebb762..f9c88d496e899 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/package.scala
@@ -17,8 +17,6 @@
package org.apache.spark.sql
-import org.apache.spark.sql.internal.SQLConf
-
/**
* Catalyst is a library for manipulating relational query plans. All classes in catalyst are
* considered an internal API to Spark SQL and are subject to change between minor releases.
@@ -30,10 +28,4 @@ package object catalyst {
* 2.10.* builds. See SI-6240 for more details.
*/
protected[sql] object ScalaReflectionLock
-
- /**
- * This class is only here to minimize the change for ticket SPARK-19944
- * (moves SQLConf from sql/core to sql/catalyst). This class should eventually be removed.
- */
- type CatalystConf = SQLConf
}
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
index 162051a8c0e4a..fab7e4c5b1285 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
@@ -75,6 +75,11 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
visitTableIdentifier(ctx.tableIdentifier)
}
+ override def visitSingleFunctionIdentifier(
+ ctx: SingleFunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) {
+ visitFunctionIdentifier(ctx.functionIdentifier)
+ }
+
override def visitSingleDataType(ctx: SingleDataTypeContext): DataType = withOrigin(ctx) {
visitSparkDataType(ctx.dataType)
}
@@ -759,6 +764,14 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] with Logging {
TableIdentifier(ctx.table.getText, Option(ctx.db).map(_.getText))
}
+ /**
+ * Create a [[FunctionIdentifier]] from a 'functionName' or 'databaseName'.'functionName' pattern.
+ */
+ override def visitFunctionIdentifier(
+ ctx: FunctionIdentifierContext): FunctionIdentifier = withOrigin(ctx) {
+ FunctionIdentifier(ctx.function.getText, Option(ctx.db).map(_.getText))
+ }
+
/* ********************************************************************************************
* Expression parsing
* ******************************************************************************************** */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
index f704b0998cada..80ab75cc17fab 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParseDriver.scala
@@ -22,7 +22,7 @@ import org.antlr.v4.runtime.misc.ParseCancellationException
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.Origin
@@ -49,6 +49,11 @@ abstract class AbstractSqlParser extends ParserInterface with Logging {
astBuilder.visitSingleTableIdentifier(parser.singleTableIdentifier())
}
+ /** Creates FunctionIdentifier for a given SQL string. */
+ def parseFunctionIdentifier(sqlText: String): FunctionIdentifier = parse(sqlText) { parser =>
+ astBuilder.visitSingleFunctionIdentifier(parser.singleFunctionIdentifier())
+ }
+
/**
* Creates StructType for a given SQL string, which is a comma separated list of field
* definitions which will preserve the correct Hive metadata.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
index 6edbe253970e9..db3598bde04d3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/ParserInterface.scala
@@ -17,7 +17,7 @@
package org.apache.spark.sql.catalyst.parser
-import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.expressions.Expression
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.types.StructType
@@ -35,6 +35,9 @@ trait ParserInterface {
/** Creates TableIdentifier for a given SQL string. */
def parseTableIdentifier(sqlText: String): TableIdentifier
+ /** Creates FunctionIdentifier for a given SQL string. */
+ def parseFunctionIdentifier(sqlText: String): FunctionIdentifier
+
/**
* Creates StructType for a given SQL string, which is a comma separated list of field
* definitions which will preserve the correct Hive metadata.
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
index 1faabcfcb73b5..b7177c4a2c4e4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LocalRelation.scala
@@ -18,9 +18,10 @@
package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions.{Attribute, Literal}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{StructField, StructType}
object LocalRelation {
@@ -74,7 +75,7 @@ case class LocalRelation(output: Seq[Attribute], data: Seq[InternalRow] = Nil)
}
}
- override def computeStats(conf: CatalystConf): Statistics =
+ override def computeStats(conf: SQLConf): Statistics =
Statistics(sizeInBytes =
output.map(n => BigInt(n.dataType.defaultSize)).sum * data.length)
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
index f71a976bd7a24..036b6256684cb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
@@ -19,11 +19,11 @@ package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.internal.Logging
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.trees.CurrentOrigin
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.StructType
@@ -90,7 +90,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
* first time. If the configuration changes, the cache can be invalidated by calling
* [[invalidateStatsCache()]].
*/
- final def stats(conf: CatalystConf): Statistics = statsCache.getOrElse {
+ final def stats(conf: SQLConf): Statistics = statsCache.getOrElse {
statsCache = Some(computeStats(conf))
statsCache.get
}
@@ -108,7 +108,7 @@ abstract class LogicalPlan extends QueryPlan[LogicalPlan] with Logging {
*
* [[LeafNode]]s must override this.
*/
- protected def computeStats(conf: CatalystConf): Statistics = {
+ protected def computeStats(conf: SQLConf): Statistics = {
if (children.isEmpty) {
throw new UnsupportedOperationException(s"LeafNode $nodeName must implement statistics.")
}
@@ -335,7 +335,7 @@ abstract class UnaryNode extends LogicalPlan {
override protected def validConstraints: Set[Expression] = child.constraints
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
// There should be some overhead in Row object, the size should not be zero when there is
// no columns, this help to prevent divide-by-zero error.
val childRowSize = child.output.map(_.dataType.defaultSize).sum + 8
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
index 19db42c80895c..c91de08ca5ef6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicLogicalOperators.scala
@@ -17,13 +17,13 @@
package org.apache.spark.sql.catalyst.plans.logical
-import org.apache.spark.sql.catalyst.{CatalystConf, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTypes}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation._
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
import org.apache.spark.util.Utils
@@ -64,7 +64,7 @@ case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extend
override def validConstraints: Set[Expression] =
child.constraints.union(getAliasedConstraints(projectList))
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
if (conf.cboEnabled) {
ProjectEstimation.estimate(conf, this).getOrElse(super.computeStats(conf))
} else {
@@ -138,7 +138,7 @@ case class Filter(condition: Expression, child: LogicalPlan)
child.constraints.union(predicates.toSet)
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
if (conf.cboEnabled) {
FilterEstimation(this, conf).estimate.getOrElse(super.computeStats(conf))
} else {
@@ -191,7 +191,7 @@ case class Intersect(left: LogicalPlan, right: LogicalPlan) extends SetOperation
}
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val leftSize = left.stats(conf).sizeInBytes
val rightSize = right.stats(conf).sizeInBytes
val sizeInBytes = if (leftSize < rightSize) leftSize else rightSize
@@ -208,7 +208,7 @@ case class Except(left: LogicalPlan, right: LogicalPlan) extends SetOperation(le
override protected def validConstraints: Set[Expression] = leftConstraints
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
left.stats(conf).copy()
}
}
@@ -247,7 +247,7 @@ case class Union(children: Seq[LogicalPlan]) extends LogicalPlan {
children.length > 1 && childrenResolved && allChildrenCompatible
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val sizeInBytes = children.map(_.stats(conf).sizeInBytes).sum
Statistics(sizeInBytes = sizeInBytes)
}
@@ -356,7 +356,7 @@ case class Join(
case _ => resolvedExceptNatural
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
def simpleEstimation: Statistics = joinType match {
case LeftAnti | LeftSemi =>
// LeftSemi and LeftAnti won't ever be bigger than left
@@ -382,7 +382,7 @@ case class BroadcastHint(child: LogicalPlan) extends UnaryNode {
override def output: Seq[Attribute] = child.output
// set isBroadcastable to true so the child will be broadcasted
- override def computeStats(conf: CatalystConf): Statistics =
+ override def computeStats(conf: SQLConf): Statistics =
child.stats(conf).copy(isBroadcastable = true)
}
@@ -538,7 +538,7 @@ case class Range(
override def newInstance(): Range = copy(output = output.map(_.newInstance()))
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val sizeInBytes = LongType.defaultSize * numElements
Statistics( sizeInBytes = sizeInBytes )
}
@@ -571,7 +571,7 @@ case class Aggregate(
child.constraints.union(getAliasedConstraints(nonAgg))
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
def simpleEstimation: Statistics = {
if (groupingExpressions.isEmpty) {
Statistics(
@@ -687,7 +687,7 @@ case class Expand(
override def references: AttributeSet =
AttributeSet(projections.flatten.flatMap(_.references))
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val sizeInBytes = super.computeStats(conf).sizeInBytes * projections.length
Statistics(sizeInBytes = sizeInBytes)
}
@@ -758,7 +758,7 @@ case class GlobalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryN
case _ => None
}
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val limit = limitExpr.eval().asInstanceOf[Int]
val childStats = child.stats(conf)
val rowCount: BigInt = childStats.rowCount.map(_.min(limit)).getOrElse(limit)
@@ -778,7 +778,7 @@ case class LocalLimit(limitExpr: Expression, child: LogicalPlan) extends UnaryNo
case _ => None
}
}
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val limit = limitExpr.eval().asInstanceOf[Int]
val childStats = child.stats(conf)
if (limit == 0) {
@@ -827,7 +827,7 @@ case class Sample(
override def output: Seq[Attribute] = child.output
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
val ratio = upperBound - lowerBound
val childStats = child.stats(conf)
var sizeInBytes = EstimationUtils.ceil(BigDecimal(childStats.sizeInBytes) * ratio)
@@ -893,7 +893,7 @@ case class RepartitionByExpression(
case object OneRowRelation extends LeafNode {
override def maxRows: Option[Long] = Some(1)
override def output: Seq[Attribute] = Nil
- override def computeStats(conf: CatalystConf): Statistics = Statistics(sizeInBytes = 1)
+ override def computeStats(conf: SQLConf): Statistics = Statistics(sizeInBytes = 1)
}
/** A logical plan for `dropDuplicates`. */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala
index ce74554c17010..48b5fbb03ef1e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/AggregateEstimation.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql.catalyst.plans.logical.statsEstimation
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Statistics}
+import org.apache.spark.sql.internal.SQLConf
object AggregateEstimation {
@@ -29,7 +29,7 @@ object AggregateEstimation {
* Estimate the number of output rows based on column stats of group-by columns, and propagate
* column stats for aggregate expressions.
*/
- def estimate(conf: CatalystConf, agg: Aggregate): Option[Statistics] = {
+ def estimate(conf: SQLConf, agg: Aggregate): Option[Statistics] = {
val childStats = agg.child.stats(conf)
// Check if we have column stats for all group-by columns.
val colStatsExist = agg.groupingExpressions.forall { e =>
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala
index 4d18b28be8663..5577233ffa6fe 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/EstimationUtils.scala
@@ -19,16 +19,16 @@ package org.apache.spark.sql.catalyst.plans.logical.statsEstimation
import scala.math.BigDecimal.RoundingMode
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan, Statistics}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{DataType, StringType}
object EstimationUtils {
/** Check if each plan has rowCount in its statistics. */
- def rowCountsExist(conf: CatalystConf, plans: LogicalPlan*): Boolean =
+ def rowCountsExist(conf: SQLConf, plans: LogicalPlan*): Boolean =
plans.forall(_.stats(conf).rowCount.isDefined)
/** Check if each attribute has column stat in the corresponding statistics. */
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
old mode 100644
new mode 100755
index b32374c5742ef..7bd8e6511232f
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/FilterEstimation.scala
@@ -22,14 +22,14 @@ import scala.collection.mutable
import scala.math.BigDecimal.RoundingMode
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.Literal.{FalseLiteral, TrueLiteral}
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Filter, LeafNode, Statistics}
import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
-case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Logging {
+case class FilterEstimation(plan: Filter, catalystConf: SQLConf) extends Logging {
private val childStats = plan.child.stats(catalystConf)
@@ -201,6 +201,21 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
case IsNotNull(ar: Attribute) if plan.child.isInstanceOf[LeafNode] =>
evaluateNullCheck(ar, isNull = false, update)
+ case op @ Equality(attrLeft: Attribute, attrRight: Attribute) =>
+ evaluateBinaryForTwoColumns(op, attrLeft, attrRight, update)
+
+ case op @ LessThan(attrLeft: Attribute, attrRight: Attribute) =>
+ evaluateBinaryForTwoColumns(op, attrLeft, attrRight, update)
+
+ case op @ LessThanOrEqual(attrLeft: Attribute, attrRight: Attribute) =>
+ evaluateBinaryForTwoColumns(op, attrLeft, attrRight, update)
+
+ case op @ GreaterThan(attrLeft: Attribute, attrRight: Attribute) =>
+ evaluateBinaryForTwoColumns(op, attrLeft, attrRight, update)
+
+ case op @ GreaterThanOrEqual(attrLeft: Attribute, attrRight: Attribute) =>
+ evaluateBinaryForTwoColumns(op, attrLeft, attrRight, update)
+
case _ =>
// TODO: it's difficult to support string operators without advanced statistics.
// Hence, these string operators Like(_, _) | Contains(_, _) | StartsWith(_, _)
@@ -257,7 +272,7 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
/**
* Returns a percentage of rows meeting a binary comparison expression.
*
- * @param op a binary comparison operator uch as =, <, <=, >, >=
+ * @param op a binary comparison operator such as =, <, <=, >, >=
* @param attr an Attribute (or a column)
* @param literal a literal value (or constant)
* @param update a boolean flag to specify if we need to update ColumnStat of a given column
@@ -448,7 +463,7 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
* Returns a percentage of rows meeting a binary comparison expression.
* This method evaluate expression for Numeric/Date/Timestamp/Boolean columns.
*
- * @param op a binary comparison operator uch as =, <, <=, >, >=
+ * @param op a binary comparison operator such as =, <, <=, >, >=
* @param attr an Attribute (or a column)
* @param literal a literal value (or constant)
* @param update a boolean flag to specify if we need to update ColumnStat of a given column
@@ -550,6 +565,220 @@ case class FilterEstimation(plan: Filter, catalystConf: CatalystConf) extends Lo
Some(percent.toDouble)
}
+ /**
+ * Returns a percentage of rows meeting a binary comparison expression containing two columns.
+ * In SQL queries, we also see predicate expressions involving two columns
+ * such as "column-1 (op) column-2" where column-1 and column-2 belong to same table.
+ * Note that, if column-1 and column-2 belong to different tables, then it is a join
+ * operator's work, NOT a filter operator's work.
+ *
+ * @param op a binary comparison operator, including =, <=>, <, <=, >, >=
+ * @param attrLeft the left Attribute (or a column)
+ * @param attrRight the right Attribute (or a column)
+ * @param update a boolean flag to specify if we need to update ColumnStat of the given columns
+ * for subsequent conditions
+ * @return an optional double value to show the percentage of rows meeting a given condition
+ */
+ def evaluateBinaryForTwoColumns(
+ op: BinaryComparison,
+ attrLeft: Attribute,
+ attrRight: Attribute,
+ update: Boolean): Option[Double] = {
+
+ if (!colStatsMap.contains(attrLeft)) {
+ logDebug("[CBO] No statistics for " + attrLeft)
+ return None
+ }
+ if (!colStatsMap.contains(attrRight)) {
+ logDebug("[CBO] No statistics for " + attrRight)
+ return None
+ }
+
+ attrLeft.dataType match {
+ case StringType | BinaryType =>
+ // TODO: It is difficult to support other binary comparisons for String/Binary
+ // type without min/max and advanced statistics like histogram.
+ logDebug("[CBO] No range comparison statistics for String/Binary type " + attrLeft)
+ return None
+ case _ =>
+ }
+
+ val colStatLeft = colStatsMap(attrLeft)
+ val statsRangeLeft = Range(colStatLeft.min, colStatLeft.max, attrLeft.dataType)
+ .asInstanceOf[NumericRange]
+ val maxLeft = BigDecimal(statsRangeLeft.max)
+ val minLeft = BigDecimal(statsRangeLeft.min)
+
+ val colStatRight = colStatsMap(attrRight)
+ val statsRangeRight = Range(colStatRight.min, colStatRight.max, attrRight.dataType)
+ .asInstanceOf[NumericRange]
+ val maxRight = BigDecimal(statsRangeRight.max)
+ val minRight = BigDecimal(statsRangeRight.min)
+
+ // determine the overlapping degree between predicate range and column's range
+ val allNotNull = (colStatLeft.nullCount == 0) && (colStatRight.nullCount == 0)
+ val (noOverlap: Boolean, completeOverlap: Boolean) = op match {
+ // Left < Right or Left <= Right
+ // - no overlap:
+ // minRight maxRight minLeft maxLeft
+ // --------+------------------+------------+-------------+------->
+ // - complete overlap: (If null values exists, we set it to partial overlap.)
+ // minLeft maxLeft minRight maxRight
+ // --------+------------------+------------+-------------+------->
+ case _: LessThan =>
+ (minLeft >= maxRight, (maxLeft < minRight) && allNotNull)
+ case _: LessThanOrEqual =>
+ (minLeft > maxRight, (maxLeft <= minRight) && allNotNull)
+
+ // Left > Right or Left >= Right
+ // - no overlap:
+ // minLeft maxLeft minRight maxRight
+ // --------+------------------+------------+-------------+------->
+ // - complete overlap: (If null values exists, we set it to partial overlap.)
+ // minRight maxRight minLeft maxLeft
+ // --------+------------------+------------+-------------+------->
+ case _: GreaterThan =>
+ (maxLeft <= minRight, (minLeft > maxRight) && allNotNull)
+ case _: GreaterThanOrEqual =>
+ (maxLeft < minRight, (minLeft >= maxRight) && allNotNull)
+
+ // Left = Right or Left <=> Right
+ // - no overlap:
+ // minLeft maxLeft minRight maxRight
+ // --------+------------------+------------+-------------+------->
+ // minRight maxRight minLeft maxLeft
+ // --------+------------------+------------+-------------+------->
+ // - complete overlap:
+ // minLeft maxLeft
+ // minRight maxRight
+ // --------+------------------+------->
+ case _: EqualTo =>
+ ((maxLeft < minRight) || (maxRight < minLeft),
+ (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
+ && (colStatLeft.distinctCount == colStatRight.distinctCount)
+ )
+ case _: EqualNullSafe =>
+ // For null-safe equality, we use a very restrictive condition to evaluate its overlap.
+ // If null values exists, we set it to partial overlap.
+ (((maxLeft < minRight) || (maxRight < minLeft)) && allNotNull,
+ (minLeft == minRight) && (maxLeft == maxRight) && allNotNull
+ && (colStatLeft.distinctCount == colStatRight.distinctCount)
+ )
+ }
+
+ var percent = BigDecimal(1.0)
+ if (noOverlap) {
+ percent = 0.0
+ } else if (completeOverlap) {
+ percent = 1.0
+ } else {
+ // For partial overlap, we use an empirical value 1/3 as suggested by the book
+ // "Database Systems, the complete book".
+ percent = 1.0 / 3.0
+
+ if (update) {
+ // Need to adjust new min/max after the filter condition is applied
+
+ val ndvLeft = BigDecimal(colStatLeft.distinctCount)
+ var newNdvLeft = (ndvLeft * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
+ if (newNdvLeft < 1) newNdvLeft = 1
+ val ndvRight = BigDecimal(colStatRight.distinctCount)
+ var newNdvRight = (ndvRight * percent).setScale(0, RoundingMode.HALF_UP).toBigInt()
+ if (newNdvRight < 1) newNdvRight = 1
+
+ var newMaxLeft = colStatLeft.max
+ var newMinLeft = colStatLeft.min
+ var newMaxRight = colStatRight.max
+ var newMinRight = colStatRight.min
+
+ op match {
+ case _: LessThan | _: LessThanOrEqual =>
+ // the left side should be less than the right side.
+ // If not, we need to adjust it to narrow the range.
+ // Left < Right or Left <= Right
+ // minRight < minLeft
+ // --------+******************+------->
+ // filtered ^
+ // |
+ // newMinRight
+ //
+ // maxRight < maxLeft
+ // --------+******************+------->
+ // ^ filtered
+ // |
+ // newMaxLeft
+ if (minLeft > minRight) newMinRight = colStatLeft.min
+ if (maxLeft > maxRight) newMaxLeft = colStatRight.max
+
+ case _: GreaterThan | _: GreaterThanOrEqual =>
+ // the left side should be greater than the right side.
+ // If not, we need to adjust it to narrow the range.
+ // Left > Right or Left >= Right
+ // minLeft < minRight
+ // --------+******************+------->
+ // filtered ^
+ // |
+ // newMinLeft
+ //
+ // maxLeft < maxRight
+ // --------+******************+------->
+ // ^ filtered
+ // |
+ // newMaxRight
+ if (minLeft < minRight) newMinLeft = colStatRight.min
+ if (maxLeft < maxRight) newMaxRight = colStatLeft.max
+
+ case _: EqualTo | _: EqualNullSafe =>
+ // need to set new min to the larger min value, and
+ // set the new max to the smaller max value.
+ // Left = Right or Left <=> Right
+ // minLeft < minRight
+ // --------+******************+------->
+ // filtered ^
+ // |
+ // newMinLeft
+ //
+ // minRight <= minLeft
+ // --------+******************+------->
+ // filtered ^
+ // |
+ // newMinRight
+ //
+ // maxLeft < maxRight
+ // --------+******************+------->
+ // ^ filtered
+ // |
+ // newMaxRight
+ //
+ // maxRight <= maxLeft
+ // --------+******************+------->
+ // ^ filtered
+ // |
+ // newMaxLeft
+ if (minLeft < minRight) {
+ newMinLeft = colStatRight.min
+ } else {
+ newMinRight = colStatLeft.min
+ }
+ if (maxLeft < maxRight) {
+ newMaxRight = colStatLeft.max
+ } else {
+ newMaxLeft = colStatRight.max
+ }
+ }
+
+ val newStatsLeft = colStatLeft.copy(distinctCount = newNdvLeft, min = newMinLeft,
+ max = newMaxLeft)
+ colStatsMap(attrLeft) = newStatsLeft
+ val newStatsRight = colStatRight.copy(distinctCount = newNdvRight, min = newMinRight,
+ max = newMaxRight)
+ colStatsMap(attrRight) = newStatsRight
+ }
+ }
+
+ Some(percent.toDouble)
+ }
+
}
class ColumnStatsMap {
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
index 9782c0bb0a939..3245a73c8a2eb 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/JoinEstimation.scala
@@ -21,12 +21,12 @@ import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
import org.apache.spark.internal.Logging
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Expression}
import org.apache.spark.sql.catalyst.planning.ExtractEquiJoinKeys
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, Statistics}
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
+import org.apache.spark.sql.internal.SQLConf
object JoinEstimation extends Logging {
@@ -34,7 +34,7 @@ object JoinEstimation extends Logging {
* Estimate statistics after join. Return `None` if the join type is not supported, or we don't
* have enough statistics for estimation.
*/
- def estimate(conf: CatalystConf, join: Join): Option[Statistics] = {
+ def estimate(conf: SQLConf, join: Join): Option[Statistics] = {
join.joinType match {
case Inner | Cross | LeftOuter | RightOuter | FullOuter =>
InnerOuterEstimation(conf, join).doEstimate()
@@ -47,7 +47,7 @@ object JoinEstimation extends Logging {
}
}
-case class InnerOuterEstimation(conf: CatalystConf, join: Join) extends Logging {
+case class InnerOuterEstimation(conf: SQLConf, join: Join) extends Logging {
private val leftStats = join.left.stats(conf)
private val rightStats = join.right.stats(conf)
@@ -288,7 +288,7 @@ case class InnerOuterEstimation(conf: CatalystConf, join: Join) extends Logging
}
}
-case class LeftSemiAntiEstimation(conf: CatalystConf, join: Join) {
+case class LeftSemiAntiEstimation(conf: SQLConf, join: Join) {
def doEstimate(): Option[Statistics] = {
// TODO: It's error-prone to estimate cardinalities for LeftSemi and LeftAnti based on basic
// column stats. Now we just propagate the statistics from left side. We should do more
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala
index e9084ad8b859c..d700cd3b20f7d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/statsEstimation/ProjectEstimation.scala
@@ -17,14 +17,14 @@
package org.apache.spark.sql.catalyst.plans.logical.statsEstimation
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap}
import org.apache.spark.sql.catalyst.plans.logical.{Project, Statistics}
+import org.apache.spark.sql.internal.SQLConf
object ProjectEstimation {
import EstimationUtils._
- def estimate(conf: CatalystConf, project: Project): Option[Statistics] = {
+ def estimate(conf: SQLConf, project: Project): Option[Statistics] = {
if (rowCountsExist(conf, project.child)) {
val childStats = project.child.stats(conf)
val inputAttrStats = childStats.attributeStats
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
index 9b94c1e2b40bb..f614965520f4a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/DateTimeUtils.scala
@@ -44,6 +44,7 @@ object DateTimeUtils {
final val JULIAN_DAY_OF_EPOCH = 2440588
final val SECONDS_PER_DAY = 60 * 60 * 24L
final val MICROS_PER_SECOND = 1000L * 1000L
+ final val MILLIS_PER_SECOND = 1000L
final val NANOS_PER_SECOND = MICROS_PER_SECOND * 1000L
final val MICROS_PER_DAY = MICROS_PER_SECOND * SECONDS_PER_DAY
@@ -237,6 +238,24 @@ object DateTimeUtils {
(day.toInt, micros * 1000L)
}
+ /*
+ * Converts the timestamp to milliseconds since epoch. In spark timestamp values have microseconds
+ * precision, so this conversion is lossy.
+ */
+ def toMillis(us: SQLTimestamp): Long = {
+ // When the timestamp is negative i.e before 1970, we need to adjust the millseconds portion.
+ // Example - 1965-01-01 10:11:12.123456 is represented as (-157700927876544) in micro precision.
+ // In millis precision the above needs to be represented as (-157700927877).
+ Math.floor(us.toDouble / MILLIS_PER_SECOND).toLong
+ }
+
+ /*
+ * Converts millseconds since epoch to SQLTimestamp.
+ */
+ def fromMillis(millis: Long): SQLTimestamp = {
+ millis * 1000L
+ }
+
/**
* Parses a given UTF8 date string to the corresponding a corresponding [[Long]] value.
* The return type is [[Option]] in order to distinguish between 0L and null. The following
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
index 5566b06aa3553..5b5d547f8fe54 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
@@ -227,6 +227,13 @@ object SQLConf {
.booleanConf
.createWithDefault(true)
+ val PARQUET_INT64_AS_TIMESTAMP_MILLIS = buildConf("spark.sql.parquet.int64AsTimestampMillis")
+ .doc("When true, timestamp values will be stored as INT64 with TIMESTAMP_MILLIS as the " +
+ "extended type. In this mode, the microsecond portion of the timestamp value will be" +
+ "truncated.")
+ .booleanConf
+ .createWithDefault(false)
+
val PARQUET_CACHE_METADATA = buildConf("spark.sql.parquet.cacheMetadata")
.doc("Turns on caching of Parquet schema metadata. Can speed up querying of static data.")
.booleanConf
@@ -935,6 +942,8 @@ class SQLConf extends Serializable with Logging {
def isParquetINT96AsTimestamp: Boolean = getConf(PARQUET_INT96_AS_TIMESTAMP)
+ def isParquetINT64AsTimestampMillis: Boolean = getConf(PARQUET_INT64_AS_TIMESTAMP_MILLIS)
+
def writeLegacyParquetFormat: Boolean = getConf(PARQUET_WRITE_LEGACY_FORMAT)
def inMemoryPartitionPruning: Boolean = getConf(IN_MEMORY_PARTITION_PRUNING)
@@ -1142,4 +1151,13 @@ class SQLConf extends Serializable with Logging {
}
result
}
+
+ // For test only
+ private[spark] def copy(entries: (ConfigEntry[_], Any)*): SQLConf = {
+ val cloned = clone()
+ entries.foreach {
+ case (entry, value) => cloned.setConfString(entry.key, value.toString)
+ }
+ cloned
+ }
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
index 0f059b9591460..1be25ec06c741 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisTest.scala
@@ -18,10 +18,10 @@
package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.internal.SQLConf
trait AnalysisTest extends PlanTest {
@@ -29,7 +29,7 @@ trait AnalysisTest extends PlanTest {
protected val caseInsensitiveAnalyzer = makeAnalyzer(caseSensitive = false)
private def makeAnalyzer(caseSensitive: Boolean): Analyzer = {
- val conf = new SimpleCatalystConf(caseSensitive)
+ val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive)
val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
catalog.createTempView("TaBlE", TestRelations.testRelation, overrideIfExists = true)
catalog.createTempView("TaBlE2", TestRelations.testRelation2, overrideIfExists = true)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
index 6995faebfa862..8f43171f309a9 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/DecimalPrecisionSuite.scala
@@ -19,7 +19,6 @@ package org.apache.spark.sql.catalyst.analysis
import org.scalatest.BeforeAndAfter
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions._
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala
index 88f68ebadc72a..2331346f325aa 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/SubstituteUnresolvedOrdinalsSuite.scala
@@ -21,7 +21,7 @@ import org.apache.spark.sql.catalyst.analysis.TestRelations.testRelation2
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions.Literal
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
+import org.apache.spark.sql.internal.SQLConf
class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest {
private lazy val a = testRelation2.output(0)
@@ -44,7 +44,7 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest {
// order by ordinal can be turned off by config
comparePlans(
- new SubstituteUnresolvedOrdinals(conf.copy(orderByOrdinal = false)).apply(plan),
+ new SubstituteUnresolvedOrdinals(conf.copy(SQLConf.ORDER_BY_ORDINAL -> false)).apply(plan),
testRelation2.orderBy(Literal(1).asc, Literal(2).asc))
}
@@ -60,7 +60,7 @@ class SubstituteUnresolvedOrdinalsSuite extends AnalysisTest {
// group by ordinal can be turned off by config
comparePlans(
- new SubstituteUnresolvedOrdinals(conf.copy(groupByOrdinal = false)).apply(plan2),
+ new SubstituteUnresolvedOrdinals(conf.copy(SQLConf.GROUP_BY_ORDINAL -> false)).apply(plan2),
testRelation2.groupBy(Literal(1), Literal(2))('a, 'b))
}
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
index 56bca73a8857a..9ba846fb25279 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalogSuite.scala
@@ -18,12 +18,13 @@
package org.apache.spark.sql.catalyst.catalog
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.{FunctionIdentifier, SimpleCatalystConf, TableIdentifier}
+import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.parser.CatalystSqlParser
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{Range, SubqueryAlias, View}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types._
class InMemorySessionCatalogSuite extends SessionCatalogSuite {
@@ -1382,7 +1383,7 @@ abstract class SessionCatalogSuite extends PlanTest {
import org.apache.spark.sql.catalyst.dsl.plans._
Seq(true, false) foreach { caseSensitive =>
- val conf = SimpleCatalystConf(caseSensitive)
+ val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive)
val catalog = new SessionCatalog(newBasicCatalog(), new SimpleFunctionRegistry, conf)
try {
val analyzer = new Analyzer(catalog, conf)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
index 802397d50e85c..e5a3e1fd374dc 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/encoders/EncoderResolutionSuite.scala
@@ -33,6 +33,10 @@ case class StringIntClass(a: String, b: Int)
case class ComplexClass(a: Long, b: StringLongClass)
+case class ArrayClass(arr: Seq[StringIntClass])
+
+case class NestedArrayClass(nestedArr: Array[ArrayClass])
+
class EncoderResolutionSuite extends PlanTest {
private val str = UTF8String.fromString("hello")
@@ -62,6 +66,54 @@ class EncoderResolutionSuite extends PlanTest {
encoder.resolveAndBind(attrs).fromRow(InternalRow(InternalRow(str, 1.toByte), 2))
}
+ test("real type doesn't match encoder schema but they are compatible: array") {
+ val encoder = ExpressionEncoder[ArrayClass]
+ val attrs = Seq('arr.array(new StructType().add("a", "int").add("b", "int").add("c", "int")))
+ val array = new GenericArrayData(Array(InternalRow(1, 2, 3)))
+ encoder.resolveAndBind(attrs).fromRow(InternalRow(array))
+ }
+
+ test("real type doesn't match encoder schema but they are compatible: nested array") {
+ val encoder = ExpressionEncoder[NestedArrayClass]
+ val et = new StructType().add("arr", ArrayType(
+ new StructType().add("a", "int").add("b", "int").add("c", "int")))
+ val attrs = Seq('nestedArr.array(et))
+ val innerArr = new GenericArrayData(Array(InternalRow(1, 2, 3)))
+ val outerArr = new GenericArrayData(Array(InternalRow(innerArr)))
+ encoder.resolveAndBind(attrs).fromRow(InternalRow(outerArr))
+ }
+
+ test("the real type is not compatible with encoder schema: non-array field") {
+ val encoder = ExpressionEncoder[ArrayClass]
+ val attrs = Seq('arr.int)
+ assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message ==
+ "need an array field but got int")
+ }
+
+ test("the real type is not compatible with encoder schema: array element type") {
+ val encoder = ExpressionEncoder[ArrayClass]
+ val attrs = Seq('arr.array(new StructType().add("c", "int")))
+ assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message ==
+ "No such struct field a in c")
+ }
+
+ test("the real type is not compatible with encoder schema: nested array element type") {
+ val encoder = ExpressionEncoder[NestedArrayClass]
+
+ withClue("inner element is not array") {
+ val attrs = Seq('nestedArr.array(new StructType().add("arr", "int")))
+ assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message ==
+ "need an array field but got int")
+ }
+
+ withClue("nested array element type is not compatible") {
+ val attrs = Seq('nestedArr.array(new StructType()
+ .add("arr", ArrayType(new StructType().add("c", "int")))))
+ assert(intercept[AnalysisException](encoder.resolveAndBind(attrs)).message ==
+ "No such struct field a in c")
+ }
+ }
+
test("nullability of array type element should not fail analysis") {
val encoder = ExpressionEncoder[Seq[Int]]
val attrs = 'a.array(IntegerType) :: Nil
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala
index b45bd977cbba1..e6132ab2e4d17 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/AggregateOptimizeSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry}
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.expressions._
@@ -26,9 +25,11 @@ import org.apache.spark.sql.catalyst.expressions.Literal
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, GROUP_BY_ORDINAL}
class AggregateOptimizeSuite extends PlanTest {
- override val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, groupByOrdinal = false)
+ override val conf = new SQLConf().copy(CASE_SENSITIVE -> false, GROUP_BY_ORDINAL -> false)
val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
val analyzer = new Analyzer(catalog, conf)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala
index 2bfddb7bc2f35..b29e1cbd14943 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BinaryComparisonSimplificationSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -30,7 +29,6 @@ import org.apache.spark.sql.catalyst.rules._
class BinaryComparisonSimplificationSuite extends PlanTest with PredicateHelper {
object Optimize extends RuleExecutor[LogicalPlan] {
- val conf = SimpleCatalystConf(caseSensitiveAnalysis = true)
val batches =
Batch("AnalysisNodes", Once,
EliminateSubqueryAliases) ::
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala
index 4d404f55aa570..935bff7cef2e8 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/BooleanSimplificationSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.expressions._
@@ -26,11 +25,11 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf
class BooleanSimplificationSuite extends PlanTest with PredicateHelper {
object Optimize extends RuleExecutor[LogicalPlan] {
- val conf = SimpleCatalystConf(caseSensitiveAnalysis = true)
val batches =
Batch("AnalysisNodes", Once,
EliminateSubqueryAliases) ::
@@ -139,7 +138,7 @@ class BooleanSimplificationSuite extends PlanTest with PredicateHelper {
checkCondition(!(('a || 'b) && ('c || 'd)), (!'a && !'b) || (!'c && !'d))
}
- private val caseInsensitiveConf = new SimpleCatalystConf(false)
+ private val caseInsensitiveConf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> false)
private val caseInsensitiveAnalyzer = new Analyzer(
new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, caseInsensitiveConf),
caseInsensitiveConf)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala
index 276b8055b08d0..ac71887c16f96 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/CombiningLimitsSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.plans.logical._
@@ -33,7 +32,7 @@ class CombiningLimitsSuite extends PlanTest {
Batch("Combine Limit", FixedPoint(10),
CombineLimits) ::
Batch("Constant Folding", FixedPoint(10),
- NullPropagation(SimpleCatalystConf(caseSensitiveAnalysis = true)),
+ NullPropagation(conf),
ConstantFolding,
BooleanSimplification,
SimplifyConditionals) :: Nil
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
index d9655bbcc2ce1..25c592b9c1dde 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedExtractValue}
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -34,7 +33,7 @@ class ConstantFoldingSuite extends PlanTest {
Batch("AnalysisNodes", Once,
EliminateSubqueryAliases) ::
Batch("ConstantFolding", Once,
- OptimizeIn(SimpleCatalystConf(true)),
+ OptimizeIn(conf),
ConstantFolding,
BooleanSimplification) :: Nil
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala
index a491f4433370d..cc4fb3a244a98 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/DecimalAggregatesSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions._
@@ -30,7 +29,7 @@ class DecimalAggregatesSuite extends PlanTest {
object Optimize extends RuleExecutor[LogicalPlan] {
val batches = Batch("Decimal Optimizations", FixedPoint(100),
- DecimalAggregates(SimpleCatalystConf(caseSensitiveAnalysis = true))) :: Nil
+ DecimalAggregates(conf)) :: Nil
}
val testRelation = LocalRelation('a.decimal(2, 1), 'b.decimal(12, 1))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala
index c5f9cc1852752..e318f36d78270 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/EliminateSortsSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry}
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.expressions._
@@ -26,9 +25,11 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, ORDER_BY_ORDINAL}
class EliminateSortsSuite extends PlanTest {
- override val conf = new SimpleCatalystConf(caseSensitiveAnalysis = true, orderByOrdinal = false)
+ override val conf = new SQLConf().copy(CASE_SENSITIVE -> true, ORDER_BY_ORDINAL -> false)
val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
val analyzer = new Analyzer(catalog, conf)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala
index 98d8b897a9165..c8fe37462726a 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/InferFiltersFromConstraintsSuite.scala
@@ -17,13 +17,13 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED
class InferFiltersFromConstraintsSuite extends PlanTest {
@@ -32,7 +32,7 @@ class InferFiltersFromConstraintsSuite extends PlanTest {
Batch("InferAndPushDownFilters", FixedPoint(100),
PushPredicateThroughJoin,
PushDownPredicate,
- InferFiltersFromConstraints(SimpleCatalystConf(caseSensitiveAnalysis = true)),
+ InferFiltersFromConstraints(conf),
CombineFilters) :: Nil
}
@@ -41,8 +41,7 @@ class InferFiltersFromConstraintsSuite extends PlanTest {
Batch("InferAndPushDownFilters", FixedPoint(100),
PushPredicateThroughJoin,
PushDownPredicate,
- InferFiltersFromConstraints(SimpleCatalystConf(caseSensitiveAnalysis = true,
- constraintPropagationEnabled = false)),
+ InferFiltersFromConstraints(conf.copy(CONSTRAINT_PROPAGATION_ENABLED -> false)),
CombineFilters) :: Nil
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala
index 61e81808147c7..a43d78c7bd447 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinOptimizationSuite.scala
@@ -26,7 +26,6 @@ import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins
import org.apache.spark.sql.catalyst.plans.{Cross, Inner, InnerLike, PlanTest}
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.RuleExecutor
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
class JoinOptimizationSuite extends PlanTest {
@@ -38,7 +37,7 @@ class JoinOptimizationSuite extends PlanTest {
CombineFilters,
PushDownPredicate,
BooleanSimplification,
- ReorderJoin(SimpleCatalystConf(true)),
+ ReorderJoin(conf),
PushPredicateThroughJoin,
ColumnPruning,
CollapseProject) :: Nil
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala
index d74008c1b3027..1922eb30fdce4 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/JoinReorderSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
@@ -25,12 +24,14 @@ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, CBO_ENABLED, JOIN_REORDER_ENABLED}
class JoinReorderSuite extends PlanTest with StatsEstimationTestBase {
- override val conf = SimpleCatalystConf(
- caseSensitiveAnalysis = true, cboEnabled = true, joinReorderEnabled = true)
+ override val conf = new SQLConf().copy(
+ CASE_SENSITIVE -> true, CBO_ENABLED -> true, JOIN_REORDER_ENABLED -> true)
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala
index 0f3ba6c895566..2885fd6841e9d 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/LimitPushdownSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala
index 4385b0e019f25..f3b65cc797ec4 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeCodegenSuite.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.Literal._
import org.apache.spark.sql.catalyst.plans.PlanTest
@@ -29,7 +28,7 @@ import org.apache.spark.sql.catalyst.rules._
class OptimizeCodegenSuite extends PlanTest {
object Optimize extends RuleExecutor[LogicalPlan] {
- val batches = Batch("OptimizeCodegen", Once, OptimizeCodegen(SimpleCatalystConf(true))) :: Nil
+ val batches = Batch("OptimizeCodegen", Once, OptimizeCodegen(conf)) :: Nil
}
protected def assertEquivalent(e1: Expression, e2: Expression): Unit = {
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
index 9daede1a5f957..d8937321ecb98 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizeInSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.{EliminateSubqueryAliases, UnresolvedAttribute}
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -25,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{Filter, LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.rules.RuleExecutor
+import org.apache.spark.sql.internal.SQLConf.OPTIMIZER_INSET_CONVERSION_THRESHOLD
import org.apache.spark.sql.types._
class OptimizeInSuite extends PlanTest {
@@ -34,10 +34,10 @@ class OptimizeInSuite extends PlanTest {
Batch("AnalysisNodes", Once,
EliminateSubqueryAliases) ::
Batch("ConstantFolding", FixedPoint(10),
- NullPropagation(SimpleCatalystConf(caseSensitiveAnalysis = true)),
+ NullPropagation(conf),
ConstantFolding,
BooleanSimplification,
- OptimizeIn(SimpleCatalystConf(caseSensitiveAnalysis = true))) :: Nil
+ OptimizeIn(conf)) :: Nil
}
val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
@@ -159,12 +159,11 @@ class OptimizeInSuite extends PlanTest {
.where(In(UnresolvedAttribute("a"), Seq(Literal(1), Literal(2), Literal(3))))
.analyze
- val notOptimizedPlan = OptimizeIn(SimpleCatalystConf(caseSensitiveAnalysis = true))(plan)
+ val notOptimizedPlan = OptimizeIn(conf)(plan)
comparePlans(notOptimizedPlan, plan)
// Reduce the threshold to turning into InSet.
- val optimizedPlan = OptimizeIn(SimpleCatalystConf(caseSensitiveAnalysis = true,
- optimizerInSetConversionThreshold = 2))(plan)
+ val optimizedPlan = OptimizeIn(conf.copy(OPTIMIZER_INSET_CONVERSION_THRESHOLD -> 2))(plan)
optimizedPlan match {
case Filter(cond, _)
if cond.isInstanceOf[InSet] && cond.asInstanceOf[InSet].getHSet().size == 3 =>
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala
index cbabc1fa6d929..b7136703b7541 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OuterJoinEliminationSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -25,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions.{Coalesce, IsNotNull}
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED
class OuterJoinEliminationSuite extends PlanTest {
object Optimize extends RuleExecutor[LogicalPlan] {
@@ -32,7 +32,7 @@ class OuterJoinEliminationSuite extends PlanTest {
Batch("Subqueries", Once,
EliminateSubqueryAliases) ::
Batch("Outer Join Elimination", Once,
- EliminateOuterJoin(SimpleCatalystConf(caseSensitiveAnalysis = true)),
+ EliminateOuterJoin(conf),
PushPredicateThroughJoin) :: Nil
}
@@ -41,8 +41,7 @@ class OuterJoinEliminationSuite extends PlanTest {
Batch("Subqueries", Once,
EliminateSubqueryAliases) ::
Batch("Outer Join Elimination", Once,
- EliminateOuterJoin(SimpleCatalystConf(caseSensitiveAnalysis = true,
- constraintPropagationEnabled = false)),
+ EliminateOuterJoin(conf.copy(CONSTRAINT_PROPAGATION_ENABLED -> false)),
PushPredicateThroughJoin) :: Nil
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
index f771e3e9eba65..c261a6091d476 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PropagateEmptyRelationSuite.scala
@@ -18,7 +18,6 @@
package org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.sql.Row
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.plans._
@@ -34,7 +33,7 @@ class PropagateEmptyRelationSuite extends PlanTest {
ReplaceExceptWithAntiJoin,
ReplaceIntersectWithSemiJoin,
PushDownPredicate,
- PruneFilters(SimpleCatalystConf(caseSensitiveAnalysis = true)),
+ PruneFilters(conf),
PropagateEmptyRelation) :: Nil
}
@@ -46,7 +45,7 @@ class PropagateEmptyRelationSuite extends PlanTest {
ReplaceExceptWithAntiJoin,
ReplaceIntersectWithSemiJoin,
PushDownPredicate,
- PruneFilters(SimpleCatalystConf(caseSensitiveAnalysis = true))) :: Nil
+ PruneFilters(conf)) :: Nil
}
val testRelation1 = LocalRelation.fromExternalRows(Seq('a.int), data = Seq(Row(1)))
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala
index 20f7f69e86c05..741dd0cf428d0 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/PruneFiltersSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -25,6 +24,7 @@ import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
+import org.apache.spark.sql.internal.SQLConf.CONSTRAINT_PROPAGATION_ENABLED
class PruneFiltersSuite extends PlanTest {
@@ -34,7 +34,7 @@ class PruneFiltersSuite extends PlanTest {
EliminateSubqueryAliases) ::
Batch("Filter Pushdown and Pruning", Once,
CombineFilters,
- PruneFilters(SimpleCatalystConf(caseSensitiveAnalysis = true)),
+ PruneFilters(conf),
PushDownPredicate,
PushPredicateThroughJoin) :: Nil
}
@@ -45,8 +45,7 @@ class PruneFiltersSuite extends PlanTest {
EliminateSubqueryAliases) ::
Batch("Filter Pushdown and Pruning", Once,
CombineFilters,
- PruneFilters(SimpleCatalystConf(caseSensitiveAnalysis = true,
- constraintPropagationEnabled = false)),
+ PruneFilters(conf.copy(CONSTRAINT_PROPAGATION_ENABLED -> false)),
PushDownPredicate,
PushPredicateThroughJoin) :: Nil
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
index 350a1c26fd1ef..8cb939e010c68 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/RewriteDistinctAggregatesSuite.scala
@@ -16,19 +16,20 @@
*/
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.{Analyzer, EmptyFunctionRegistry}
import org.apache.spark.sql.catalyst.catalog.{InMemoryCatalog, SessionCatalog}
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
-import org.apache.spark.sql.catalyst.expressions.{If, Literal}
-import org.apache.spark.sql.catalyst.expressions.aggregate.{CollectSet, Count}
+import org.apache.spark.sql.catalyst.expressions.Literal
+import org.apache.spark.sql.catalyst.expressions.aggregate.CollectSet
import org.apache.spark.sql.catalyst.plans.PlanTest
import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, Expand, LocalRelation, LogicalPlan}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, GROUP_BY_ORDINAL}
import org.apache.spark.sql.types.{IntegerType, StringType}
class RewriteDistinctAggregatesSuite extends PlanTest {
- override val conf = SimpleCatalystConf(caseSensitiveAnalysis = false, groupByOrdinal = false)
+ override val conf = new SQLConf().copy(CASE_SENSITIVE -> false, GROUP_BY_ORDINAL -> false)
val catalog = new SessionCatalog(new InMemoryCatalog, EmptyFunctionRegistry, conf)
val analyzer = new Analyzer(catalog, conf)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala
index ca4976f0d6db0..756e0f35b2178 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/SetOperationSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
@@ -35,7 +34,7 @@ class SetOperationSuite extends PlanTest {
CombineUnions,
PushProjectionThroughUnion,
PushDownPredicate,
- PruneFilters(SimpleCatalystConf(caseSensitiveAnalysis = true))) :: Nil
+ PruneFilters(conf)) :: Nil
}
val testRelation = LocalRelation('a.int, 'b.int, 'c.int)
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala
index 93fdd98d1ac93..003ce49eaf8e6 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinReorderSuite.scala
@@ -17,7 +17,6 @@
package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.dsl.plans._
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap}
@@ -25,12 +24,12 @@ import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest}
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan}
-
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, STARSCHEMA_DETECTION}
class StarJoinReorderSuite extends PlanTest with StatsEstimationTestBase {
- override val conf = SimpleCatalystConf(
- caseSensitiveAnalysis = true, starSchemaDetection = true)
+ override val conf = new SQLConf().copy(CASE_SENSITIVE -> true, STARSCHEMA_DETECTION -> true)
object Optimize extends RuleExecutor[LogicalPlan] {
val batches =
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
index c73dfaf3f8fe3..f44428c3512a9 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/plans/PlanTest.scala
@@ -18,18 +18,18 @@
package org.apache.spark.sql.catalyst.plans
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.util._
+import org.apache.spark.sql.internal.SQLConf
/**
* Provides helper methods for comparing plans.
*/
abstract class PlanTest extends SparkFunSuite with PredicateHelper {
- protected val conf = SimpleCatalystConf(caseSensitiveAnalysis = true)
+ protected val conf = new SQLConf().copy(SQLConf.CASE_SENSITIVE -> true)
/**
* Since attribute references are given globally unique ids during analysis,
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala
index c0b9515ca7cd0..38483a298cef0 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/AggregateEstimationSuite.scala
@@ -21,6 +21,7 @@ import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap
import org.apache.spark.sql.catalyst.expressions.aggregate.Count
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.logical.statsEstimation.EstimationUtils._
+import org.apache.spark.sql.internal.SQLConf
class AggregateEstimationSuite extends StatsEstimationTestBase {
@@ -101,13 +102,13 @@ class AggregateEstimationSuite extends StatsEstimationTestBase {
val noGroupAgg = Aggregate(groupingExpressions = Nil,
aggregateExpressions = Seq(Alias(Count(Literal(1)), "cnt")()), child)
- assert(noGroupAgg.stats(conf.copy(cboEnabled = false)) ==
+ assert(noGroupAgg.stats(conf.copy(SQLConf.CBO_ENABLED -> false)) ==
// overhead + count result size
Statistics(sizeInBytes = 8 + 8, rowCount = Some(1)))
val hasGroupAgg = Aggregate(groupingExpressions = attributes,
aggregateExpressions = attributes :+ Alias(Count(Literal(1)), "cnt")(), child)
- assert(hasGroupAgg.stats(conf.copy(cboEnabled = false)) ==
+ assert(hasGroupAgg.stats(conf.copy(SQLConf.CBO_ENABLED -> false)) ==
// From UnaryNode.computeStats, childSize * outputRowSize / childRowSize
Statistics(sizeInBytes = 48 * (8 + 4 + 8) / (8 + 4)))
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
index 0d92c1e35565a..b06871f96f0d8 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/BasicStatsEstimationSuite.scala
@@ -17,9 +17,9 @@
package org.apache.spark.sql.catalyst.statsEstimation
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference, Literal}
import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.IntegerType
@@ -116,10 +116,10 @@ class BasicStatsEstimationSuite extends StatsEstimationTestBase {
expectedStatsCboOff: Statistics): Unit = {
// Invalidate statistics
plan.invalidateStatsCache()
- assert(plan.stats(conf.copy(cboEnabled = true)) == expectedStatsCboOn)
+ assert(plan.stats(conf.copy(SQLConf.CBO_ENABLED -> true)) == expectedStatsCboOn)
plan.invalidateStatsCache()
- assert(plan.stats(conf.copy(cboEnabled = false)) == expectedStatsCboOff)
+ assert(plan.stats(conf.copy(SQLConf.CBO_ENABLED -> false)) == expectedStatsCboOff)
}
/** Check estimated stats when it's the same whether cbo is turned on or off. */
@@ -136,6 +136,6 @@ private case class DummyLogicalPlan(
cboStats: Statistics) extends LogicalPlan {
override def output: Seq[Attribute] = Nil
override def children: Seq[LogicalPlan] = Nil
- override def computeStats(conf: CatalystConf): Statistics =
+ override def computeStats(conf: SQLConf): Statistics =
if (conf.cboEnabled) cboStats else defaultStats
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
old mode 100644
new mode 100755
index 1966c96c05294..cffb0d8739287
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/FilterEstimationSuite.scala
@@ -33,49 +33,74 @@ import org.apache.spark.sql.types._
class FilterEstimationSuite extends StatsEstimationTestBase {
// Suppose our test table has 10 rows and 6 columns.
- // First column cint has values: 1, 2, 3, 4, 5, 6, 7, 8, 9, 10
+ // column cint has values: 1, 2, 3, 4, 5, 6, 7, 8, 9, 10
// Hence, distinctCount:10, min:1, max:10, nullCount:0, avgLen:4, maxLen:4
val attrInt = AttributeReference("cint", IntegerType)()
val colStatInt = ColumnStat(distinctCount = 10, min = Some(1), max = Some(10),
nullCount = 0, avgLen = 4, maxLen = 4)
- // only 2 values
+ // column cbool has only 2 distinct values
val attrBool = AttributeReference("cbool", BooleanType)()
val colStatBool = ColumnStat(distinctCount = 2, min = Some(false), max = Some(true),
nullCount = 0, avgLen = 1, maxLen = 1)
- // Second column cdate has 10 values from 2017-01-01 through 2017-01-10.
+ // column cdate has 10 values from 2017-01-01 through 2017-01-10.
val dMin = Date.valueOf("2017-01-01")
val dMax = Date.valueOf("2017-01-10")
val attrDate = AttributeReference("cdate", DateType)()
val colStatDate = ColumnStat(distinctCount = 10, min = Some(dMin), max = Some(dMax),
nullCount = 0, avgLen = 4, maxLen = 4)
- // Fourth column cdecimal has 4 values from 0.20 through 0.80 at increment of 0.20.
+ // column cdecimal has 4 values from 0.20 through 0.80 at increment of 0.20.
val decMin = new java.math.BigDecimal("0.200000000000000000")
val decMax = new java.math.BigDecimal("0.800000000000000000")
val attrDecimal = AttributeReference("cdecimal", DecimalType(18, 18))()
val colStatDecimal = ColumnStat(distinctCount = 4, min = Some(decMin), max = Some(decMax),
nullCount = 0, avgLen = 8, maxLen = 8)
- // Fifth column cdouble has 10 double values: 1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 9.0, 10.0
+ // column cdouble has 10 double values: 1.0, 2.0, 3.0, 4.0, 5.0, 6.0, 7.0, 8.0, 9.0, 10.0
val attrDouble = AttributeReference("cdouble", DoubleType)()
val colStatDouble = ColumnStat(distinctCount = 10, min = Some(1.0), max = Some(10.0),
nullCount = 0, avgLen = 8, maxLen = 8)
- // Sixth column cstring has 10 String values:
+ // column cstring has 10 String values:
// "A0", "A1", "A2", "A3", "A4", "A5", "A6", "A7", "A8", "A9"
val attrString = AttributeReference("cstring", StringType)()
val colStatString = ColumnStat(distinctCount = 10, min = None, max = None,
nullCount = 0, avgLen = 2, maxLen = 2)
+ // column cint2 has values: 7, 8, 9, 10, 11, 12, 13, 14, 15, 16
+ // Hence, distinctCount:10, min:7, max:16, nullCount:0, avgLen:4, maxLen:4
+ // This column is created to test "cint < cint2
+ val attrInt2 = AttributeReference("cint2", IntegerType)()
+ val colStatInt2 = ColumnStat(distinctCount = 10, min = Some(7), max = Some(16),
+ nullCount = 0, avgLen = 4, maxLen = 4)
+
+ // column cint3 has values: 30, 31, 32, 33, 34, 35, 36, 37, 38, 39
+ // Hence, distinctCount:10, min:30, max:39, nullCount:0, avgLen:4, maxLen:4
+ // This column is created to test "cint = cint3 without overlap at all.
+ val attrInt3 = AttributeReference("cint3", IntegerType)()
+ val colStatInt3 = ColumnStat(distinctCount = 10, min = Some(30), max = Some(39),
+ nullCount = 0, avgLen = 4, maxLen = 4)
+
+ // column cint4 has values in the range from 1 to 10
+ // distinctCount:10, min:1, max:10, nullCount:0, avgLen:4, maxLen:4
+ // This column is created to test complete overlap
+ val attrInt4 = AttributeReference("cint4", IntegerType)()
+ val colStatInt4 = ColumnStat(distinctCount = 10, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4)
+
val attributeMap = AttributeMap(Seq(
attrInt -> colStatInt,
attrBool -> colStatBool,
attrDate -> colStatDate,
attrDecimal -> colStatDecimal,
attrDouble -> colStatDouble,
- attrString -> colStatString))
+ attrString -> colStatString,
+ attrInt2 -> colStatInt2,
+ attrInt3 -> colStatInt3,
+ attrInt4 -> colStatInt4
+ ))
test("true") {
validateEstimatedStats(
@@ -450,6 +475,89 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
}
}
+ test("cint = cint2") {
+ // partial overlap case
+ validateEstimatedStats(
+ Filter(EqualTo(attrInt, attrInt2), childStatsTestPlan(Seq(attrInt, attrInt2), 10L)),
+ Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4),
+ attrInt2 -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4)),
+ expectedRowCount = 4)
+ }
+
+ test("cint > cint2") {
+ // partial overlap case
+ validateEstimatedStats(
+ Filter(GreaterThan(attrInt, attrInt2), childStatsTestPlan(Seq(attrInt, attrInt2), 10L)),
+ Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4),
+ attrInt2 -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4)),
+ expectedRowCount = 4)
+ }
+
+ test("cint < cint2") {
+ // partial overlap case
+ validateEstimatedStats(
+ Filter(LessThan(attrInt, attrInt2), childStatsTestPlan(Seq(attrInt, attrInt2), 10L)),
+ Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4),
+ attrInt2 -> ColumnStat(distinctCount = 3, min = Some(7), max = Some(16),
+ nullCount = 0, avgLen = 4, maxLen = 4)),
+ expectedRowCount = 4)
+ }
+
+ test("cint = cint4") {
+ // complete overlap case
+ validateEstimatedStats(
+ Filter(EqualTo(attrInt, attrInt4), childStatsTestPlan(Seq(attrInt, attrInt4), 10L)),
+ Seq(attrInt -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4),
+ attrInt4 -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4)),
+ expectedRowCount = 10)
+ }
+
+ test("cint < cint4") {
+ // partial overlap case
+ validateEstimatedStats(
+ Filter(LessThan(attrInt, attrInt4), childStatsTestPlan(Seq(attrInt, attrInt4), 10L)),
+ Seq(attrInt -> ColumnStat(distinctCount = 3, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4),
+ attrInt4 -> ColumnStat(distinctCount = 3, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4)),
+ expectedRowCount = 4)
+ }
+
+ test("cint = cint3") {
+ // no records qualify due to no overlap
+ val emptyColStats = Seq[(Attribute, ColumnStat)]()
+ validateEstimatedStats(
+ Filter(EqualTo(attrInt, attrInt3), childStatsTestPlan(Seq(attrInt, attrInt3), 10L)),
+ Nil, // set to empty
+ expectedRowCount = 0)
+ }
+
+ test("cint < cint3") {
+ // all table records qualify.
+ validateEstimatedStats(
+ Filter(LessThan(attrInt, attrInt3), childStatsTestPlan(Seq(attrInt, attrInt3), 10L)),
+ Seq(attrInt -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(10),
+ nullCount = 0, avgLen = 4, maxLen = 4),
+ attrInt3 -> ColumnStat(distinctCount = 10, min = Some(30), max = Some(39),
+ nullCount = 0, avgLen = 4, maxLen = 4)),
+ expectedRowCount = 10)
+ }
+
+ test("cint > cint3") {
+ // no records qualify due to no overlap
+ validateEstimatedStats(
+ Filter(GreaterThan(attrInt, attrInt3), childStatsTestPlan(Seq(attrInt, attrInt3), 10L)),
+ Nil, // set to empty
+ expectedRowCount = 0)
+ }
+
private def childStatsTestPlan(outList: Seq[Attribute], tableRowCount: BigInt): StatsTestPlan = {
StatsTestPlan(
outputList = outList,
@@ -491,7 +599,23 @@ class FilterEstimationSuite extends StatsEstimationTestBase {
sizeInBytes = getOutputSize(filter.output, expectedRowCount, expectedAttributeMap),
rowCount = Some(expectedRowCount),
attributeStats = expectedAttributeMap)
- assert(filter.stats(conf) == expectedStats)
+
+ val filterStats = filter.stats(conf)
+ assert(filterStats.sizeInBytes == expectedStats.sizeInBytes)
+ assert(filterStats.rowCount == expectedStats.rowCount)
+ val rowCountValue = filterStats.rowCount.getOrElse(0)
+ // check the output column stats if the row count is > 0.
+ // When row count is 0, the output is set to empty.
+ if (rowCountValue != 0) {
+ // Need to check attributeStats one by one because we may have multiple output columns.
+ // Due to update operation, the output columns may be in different order.
+ assert(expectedColStats.size == filterStats.attributeStats.size)
+ expectedColStats.foreach { kv =>
+ val filterColumnStat = filterStats.attributeStats.get(kv._1).get
+ assert(filterColumnStat == kv._2)
+ }
+ }
}
}
+
}
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala
index 9b2b8dbe1bf4a..263f4e18803d5 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/statsEstimation/StatsEstimationTestBase.scala
@@ -18,16 +18,17 @@
package org.apache.spark.sql.catalyst.statsEstimation
import org.apache.spark.SparkFunSuite
-import org.apache.spark.sql.catalyst.{CatalystConf, SimpleCatalystConf}
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LeafNode, LogicalPlan, Statistics}
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.internal.SQLConf.{CASE_SENSITIVE, CBO_ENABLED}
import org.apache.spark.sql.types.{IntegerType, StringType}
trait StatsEstimationTestBase extends SparkFunSuite {
/** Enable stats estimation based on CBO. */
- protected val conf = SimpleCatalystConf(caseSensitiveAnalysis = true, cboEnabled = true)
+ protected val conf = new SQLConf().copy(CASE_SENSITIVE -> true, CBO_ENABLED -> true)
def getColSize(attribute: Attribute, colStat: ColumnStat): Long = attribute.dataType match {
// For UTF8String: base + offset + numBytes
@@ -54,7 +55,7 @@ case class StatsTestPlan(
attributeStats: AttributeMap[ColumnStat],
size: Option[BigInt] = None) extends LeafNode {
override def output: Seq[Attribute] = outputList
- override def computeStats(conf: CatalystConf): Statistics = Statistics(
+ override def computeStats(conf: SQLConf): Statistics = Statistics(
// If sizeInBytes is useless in testing, we just use a fake value
sizeInBytes = size.getOrElse(Int.MaxValue),
rowCount = Some(rowCount),
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
index bf8717483575f..eb97118872ea1 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/SpecificParquetRecordReaderBase.java
@@ -197,6 +197,7 @@ protected void initialize(String path, List columns) throws IOException
config.set("spark.sql.parquet.binaryAsString", "false");
config.set("spark.sql.parquet.int96AsTimestamp", "false");
config.set("spark.sql.parquet.writeLegacyFormat", "false");
+ config.set("spark.sql.parquet.int64AsTimestampMillis", "false");
this.file = new Path(path);
long length = this.file.getFileSystem(config).getFileStatus(this.file).getLen();
diff --git a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
index cb51cb499eede..9d641b528723a 100644
--- a/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
+++ b/sql/core/src/main/java/org/apache/spark/sql/execution/datasources/parquet/VectorizedColumnReader.java
@@ -28,6 +28,7 @@
import org.apache.parquet.io.api.Binary;
import org.apache.parquet.schema.PrimitiveType;
+import org.apache.spark.sql.catalyst.util.DateTimeUtils;
import org.apache.spark.sql.execution.vectorized.ColumnVector;
import org.apache.spark.sql.types.DataTypes;
import org.apache.spark.sql.types.DecimalType;
@@ -155,9 +156,13 @@ void readBatch(int total, ColumnVector column) throws IOException {
// Read and decode dictionary ids.
defColumn.readIntegers(
num, dictionaryIds, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn);
+
+ // Timestamp values encoded as INT64 can't be lazily decoded as we need to post process
+ // the values to add microseconds precision.
if (column.hasDictionary() || (rowId == 0 &&
(descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT32 ||
- descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64 ||
+ (descriptor.getType() == PrimitiveType.PrimitiveTypeName.INT64 &&
+ column.dataType() != DataTypes.TimestampType) ||
descriptor.getType() == PrimitiveType.PrimitiveTypeName.FLOAT ||
descriptor.getType() == PrimitiveType.PrimitiveTypeName.DOUBLE ||
descriptor.getType() == PrimitiveType.PrimitiveTypeName.BINARY))) {
@@ -250,7 +255,15 @@ private void decodeDictionaryIds(int rowId, int num, ColumnVector column,
column.putLong(i, dictionary.decodeToLong(dictionaryIds.getDictId(i)));
}
}
- } else {
+ } else if (column.dataType() == DataTypes.TimestampType) {
+ for (int i = rowId; i < rowId + num; ++i) {
+ if (!column.isNullAt(i)) {
+ column.putLong(i,
+ DateTimeUtils.fromMillis(dictionary.decodeToLong(dictionaryIds.getDictId(i))));
+ }
+ }
+ }
+ else {
throw new UnsupportedOperationException("Unimplemented type: " + column.dataType());
}
break;
@@ -362,7 +375,15 @@ private void readLongBatch(int rowId, int num, ColumnVector column) throws IOExc
if (column.dataType() == DataTypes.LongType ||
DecimalType.is64BitDecimalType(column.dataType())) {
defColumn.readLongs(
- num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn);
+ num, column, rowId, maxDefLevel, (VectorizedValuesReader) dataColumn);
+ } else if (column.dataType() == DataTypes.TimestampType) {
+ for (int i = 0; i < num; i++) {
+ if (defColumn.readInteger() == maxDefLevel) {
+ column.putLong(rowId + i, DateTimeUtils.fromMillis(dataColumn.readLong()));
+ } else {
+ column.putNull(rowId + i);
+ }
+ }
} else {
throw new UnsupportedOperationException("Unsupported conversion to: " + column.dataType());
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
index b60499253c42f..95f3463dfe62b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala
@@ -591,8 +591,13 @@ class SparkSession private(
@transient lazy val catalog: Catalog = new CatalogImpl(self)
/**
- * Returns the specified table as a `DataFrame`.
+ * Returns the specified table/view as a `DataFrame`.
*
+ * @param tableName is either a qualified or unqualified name that designates a table or view.
+ * If a database is specified, it identifies the table/view from the database.
+ * Otherwise, it first attempts to find a temporary view with the given name
+ * and then match the table/view from the current database.
+ * Note that, the global temporary view database is also valid here.
* @since 2.0.0
*/
def table(tableName: String): DataFrame = {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
index 50252db789d46..137b0cbc84f8f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/catalog/Catalog.scala
@@ -54,16 +54,16 @@ abstract class Catalog {
def listDatabases(): Dataset[Database]
/**
- * Returns a list of tables in the current database.
- * This includes all temporary tables.
+ * Returns a list of tables/views in the current database.
+ * This includes all temporary views.
*
* @since 2.0.0
*/
def listTables(): Dataset[Table]
/**
- * Returns a list of tables in the specified database.
- * This includes all temporary tables.
+ * Returns a list of tables/views in the specified database.
+ * This includes all temporary views.
*
* @since 2.0.0
*/
@@ -88,17 +88,21 @@ abstract class Catalog {
def listFunctions(dbName: String): Dataset[Function]
/**
- * Returns a list of columns for the given table in the current database or
- * the given temporary table.
+ * Returns a list of columns for the given table/view or temporary view.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a temporary view or
+ * a table/view in the current database.
* @since 2.0.0
*/
@throws[AnalysisException]("table does not exist")
def listColumns(tableName: String): Dataset[Column]
/**
- * Returns a list of columns for the given table in the specified database.
+ * Returns a list of columns for the given table/view in the specified database.
*
+ * @param dbName is a name that designates a database.
+ * @param tableName is an unqualified name that designates a table/view.
* @since 2.0.0
*/
@throws[AnalysisException]("database or table does not exist")
@@ -115,9 +119,11 @@ abstract class Catalog {
/**
* Get the table or view with the specified name. This table can be a temporary view or a
- * table/view in the current database. This throws an AnalysisException when no Table
- * can be found.
+ * table/view. This throws an AnalysisException when no Table can be found.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a table/view in
+ * the current database.
* @since 2.1.0
*/
@throws[AnalysisException]("table does not exist")
@@ -134,9 +140,11 @@ abstract class Catalog {
/**
* Get the function with the specified name. This function can be a temporary function or a
- * function in the current database. This throws an AnalysisException when the function cannot
- * be found.
+ * function. This throws an AnalysisException when the function cannot be found.
*
+ * @param functionName is either a qualified or unqualified name that designates a function.
+ * If no database identifier is provided, it refers to a temporary function
+ * or a function in the current database.
* @since 2.1.0
*/
@throws[AnalysisException]("function does not exist")
@@ -146,6 +154,8 @@ abstract class Catalog {
* Get the function with the specified name. This throws an AnalysisException when the function
* cannot be found.
*
+ * @param dbName is a name that designates a database.
+ * @param functionName is an unqualified name that designates a function in the specified database
* @since 2.1.0
*/
@throws[AnalysisException]("database or function does not exist")
@@ -160,8 +170,11 @@ abstract class Catalog {
/**
* Check if the table or view with the specified name exists. This can either be a temporary
- * view or a table/view in the current database.
+ * view or a table/view.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a table/view in
+ * the current database.
* @since 2.1.0
*/
def tableExists(tableName: String): Boolean
@@ -169,14 +182,19 @@ abstract class Catalog {
/**
* Check if the table or view with the specified name exists in the specified database.
*
+ * @param dbName is a name that designates a database.
+ * @param tableName is an unqualified name that designates a table.
* @since 2.1.0
*/
def tableExists(dbName: String, tableName: String): Boolean
/**
* Check if the function with the specified name exists. This can either be a temporary function
- * or a function in the current database.
+ * or a function.
*
+ * @param functionName is either a qualified or unqualified name that designates a function.
+ * If no database identifier is provided, it refers to a function in
+ * the current database.
* @since 2.1.0
*/
def functionExists(functionName: String): Boolean
@@ -184,6 +202,8 @@ abstract class Catalog {
/**
* Check if the function with the specified name exists in the specified database.
*
+ * @param dbName is a name that designates a database.
+ * @param functionName is an unqualified name that designates a function.
* @since 2.1.0
*/
def functionExists(dbName: String, functionName: String): Boolean
@@ -192,6 +212,9 @@ abstract class Catalog {
* Creates a table from the given path and returns the corresponding DataFrame.
* It will use the default data source configured by spark.sql.sources.default.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.0.0
*/
@deprecated("use createTable instead.", "2.2.0")
@@ -204,6 +227,9 @@ abstract class Catalog {
* Creates a table from the given path and returns the corresponding DataFrame.
* It will use the default data source configured by spark.sql.sources.default.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.2.0
*/
@Experimental
@@ -214,6 +240,9 @@ abstract class Catalog {
* Creates a table from the given path based on a data source and returns the corresponding
* DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.0.0
*/
@deprecated("use createTable instead.", "2.2.0")
@@ -226,6 +255,9 @@ abstract class Catalog {
* Creates a table from the given path based on a data source and returns the corresponding
* DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.2.0
*/
@Experimental
@@ -236,6 +268,9 @@ abstract class Catalog {
* Creates a table from the given path based on a data source and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.0.0
*/
@deprecated("use createTable instead.", "2.2.0")
@@ -251,6 +286,9 @@ abstract class Catalog {
* Creates a table from the given path based on a data source and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.2.0
*/
@Experimental
@@ -267,6 +305,9 @@ abstract class Catalog {
* Creates a table from the given path based on a data source and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.0.0
*/
@deprecated("use createTable instead.", "2.2.0")
@@ -283,6 +324,9 @@ abstract class Catalog {
* Creates a table from the given path based on a data source and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.2.0
*/
@Experimental
@@ -297,6 +341,9 @@ abstract class Catalog {
* Create a table from the given path based on a data source, a schema and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.0.0
*/
@deprecated("use createTable instead.", "2.2.0")
@@ -313,6 +360,9 @@ abstract class Catalog {
* Create a table from the given path based on a data source, a schema and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.2.0
*/
@Experimental
@@ -330,6 +380,9 @@ abstract class Catalog {
* Create a table from the given path based on a data source, a schema and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.0.0
*/
@deprecated("use createTable instead.", "2.2.0")
@@ -347,6 +400,9 @@ abstract class Catalog {
* Create a table from the given path based on a data source, a schema and a set of options.
* Then, returns the corresponding DataFrame.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in
+ * the current database.
* @since 2.2.0
*/
@Experimental
@@ -368,7 +424,7 @@ abstract class Catalog {
* Note that, the return type of this method was Unit in Spark 2.0, but changed to Boolean
* in Spark 2.1.
*
- * @param viewName the name of the view to be dropped.
+ * @param viewName the name of the temporary view to be dropped.
* @return true if the view is dropped successfully, false otherwise.
* @since 2.0.0
*/
@@ -383,15 +439,18 @@ abstract class Catalog {
* preserved database `global_temp`, and we must use the qualified name to refer a global temp
* view, e.g. `SELECT * FROM global_temp.view1`.
*
- * @param viewName the name of the view to be dropped.
+ * @param viewName the unqualified name of the temporary view to be dropped.
* @return true if the view is dropped successfully, false otherwise.
* @since 2.1.0
*/
def dropGlobalTempView(viewName: String): Boolean
/**
- * Recover all the partitions in the directory of a table and update the catalog.
+ * Recovers all the partitions in the directory of a table and update the catalog.
*
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in the
+ * current database.
* @since 2.1.1
*/
def recoverPartitions(tableName: String): Unit
@@ -399,6 +458,9 @@ abstract class Catalog {
/**
* Returns true if the table is currently cached in-memory.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a temporary view or
+ * a table/view in the current database.
* @since 2.0.0
*/
def isCached(tableName: String): Boolean
@@ -406,6 +468,9 @@ abstract class Catalog {
/**
* Caches the specified table in-memory.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a temporary view or
+ * a table/view in the current database.
* @since 2.0.0
*/
def cacheTable(tableName: String): Unit
@@ -413,6 +478,9 @@ abstract class Catalog {
/**
* Removes the specified table from the in-memory cache.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a temporary view or
+ * a table/view in the current database.
* @since 2.0.0
*/
def uncacheTable(tableName: String): Unit
@@ -425,7 +493,7 @@ abstract class Catalog {
def clearCache(): Unit
/**
- * Invalidate and refresh all the cached metadata of the given table. For performance reasons,
+ * Invalidates and refreshes all the cached metadata of the given table. For performance reasons,
* Spark SQL or the external data source library it uses might cache certain metadata about a
* table, such as the location of blocks. When those change outside of Spark SQL, users should
* call this function to invalidate the cache.
@@ -433,13 +501,16 @@ abstract class Catalog {
* If this table is cached as an InMemoryRelation, drop the original cached version and make the
* new version cached lazily.
*
+ * @param tableName is either a qualified or unqualified name that designates a table/view.
+ * If no database identifier is provided, it refers to a temporary view or
+ * a table/view in the current database.
* @since 2.0.0
*/
def refreshTable(tableName: String): Unit
/**
- * Invalidate and refresh all the cached data (and the associated metadata) for any dataframe that
- * contains the given data source path. Path matching is by prefix, i.e. "/" would invalidate
+ * Invalidates and refreshes all the cached data (and the associated metadata) for any [[Dataset]]
+ * that contains the given data source path. Path matching is by prefix, i.e. "/" would invalidate
* everything that is cached.
*
* @since 2.0.0
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
index 49336f424822f..2827b8ac00331 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/ExistingRDD.scala
@@ -19,12 +19,13 @@ package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.{Encoder, Row, SparkSession}
-import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.plans.physical.{Partitioning, UnknownPartitioning}
import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.DataType
import org.apache.spark.util.Utils
@@ -95,7 +96,7 @@ case class ExternalRDD[T](
override protected def stringArgs: Iterator[Any] = Iterator(output)
- @transient override def computeStats(conf: CatalystConf): Statistics = Statistics(
+ @transient override def computeStats(conf: SQLConf): Statistics = Statistics(
// TODO: Instead of returning a default value here, find a way to return a meaningful size
// estimate for RDDs. See PR 1238 for more discussions.
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
@@ -170,7 +171,7 @@ case class LogicalRDD(
override protected def stringArgs: Iterator[Any] = Iterator(output)
- @transient override def computeStats(conf: CatalystConf): Statistics = Statistics(
+ @transient override def computeStats(conf: SQLConf): Statistics = Statistics(
// TODO: Instead of returning a default value here, find a way to return a meaningful size
// estimate for RDDs. See PR 1238 for more discussions.
sizeInBytes = BigInt(session.sessionState.conf.defaultSizeInBytes)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
index d4f23f9dd5185..80afb59b3e88e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlParser.scala
@@ -322,8 +322,7 @@ class SparkSqlAstBuilder(conf: SQLConf) extends AstBuilder {
DescribeTableCommand(
visitTableIdentifier(ctx.tableIdentifier),
partitionSpec,
- ctx.EXTENDED != null,
- ctx.FORMATTED != null)
+ ctx.EXTENDED != null || ctx.FORMATTED != null)
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
index 36037ac003728..0a9f3e799990f 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/columnar/InMemoryRelation.scala
@@ -21,12 +21,13 @@ import org.apache.commons.lang3.StringUtils
import org.apache.spark.network.util.JavaUtils
import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.{CatalystConf, InternalRow}
+import org.apache.spark.sql.catalyst.InternalRow
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical.Statistics
import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.storage.StorageLevel
import org.apache.spark.util.LongAccumulator
@@ -69,7 +70,7 @@ case class InMemoryRelation(
@transient val partitionStatistics = new PartitionStatistics(output)
- override def computeStats(conf: CatalystConf): Statistics = {
+ override def computeStats(conf: SQLConf): Statistics = {
if (batchStats.value == 0L) {
// Underlying columnar RDD hasn't been materialized, no useful statistics information
// available, return the default statistics.
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
index c7aeef06a0bf0..ebf03e1bf8869 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala
@@ -500,8 +500,7 @@ case class TruncateTableCommand(
case class DescribeTableCommand(
table: TableIdentifier,
partitionSpec: TablePartitionSpec,
- isExtended: Boolean,
- isFormatted: Boolean)
+ isExtended: Boolean)
extends RunnableCommand {
override val output: Seq[Attribute] = Seq(
@@ -536,14 +535,12 @@ case class DescribeTableCommand(
describePartitionInfo(metadata, result)
- if (partitionSpec.isEmpty) {
- if (isExtended) {
- describeExtendedTableInfo(metadata, result)
- } else if (isFormatted) {
- describeFormattedTableInfo(metadata, result)
- }
- } else {
+ if (partitionSpec.nonEmpty) {
+ // Outputs the partition-specific info for the DDL command:
+ // "DESCRIBE [EXTENDED|FORMATTED] table_name PARTITION (partitionVal*)"
describeDetailedPartitionInfo(sparkSession, catalog, metadata, result)
+ } else if (isExtended) {
+ describeFormattedTableInfo(metadata, result)
}
}
@@ -553,76 +550,20 @@ case class DescribeTableCommand(
private def describePartitionInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
if (table.partitionColumnNames.nonEmpty) {
append(buffer, "# Partition Information", "", "")
- append(buffer, s"# ${output.head.name}", output(1).name, output(2).name)
describeSchema(table.partitionSchema, buffer)
}
}
- private def describeExtendedTableInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
- append(buffer, "", "", "")
- append(buffer, "# Detailed Table Information", table.toString, "")
- }
-
private def describeFormattedTableInfo(table: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
+ // The following information has been already shown in the previous outputs
+ val excludedTableInfo = Seq(
+ "Partition Columns",
+ "Schema"
+ )
append(buffer, "", "", "")
append(buffer, "# Detailed Table Information", "", "")
- append(buffer, "Database:", table.database, "")
- append(buffer, "Owner:", table.owner, "")
- append(buffer, "Created:", new Date(table.createTime).toString, "")
- append(buffer, "Last Access:", new Date(table.lastAccessTime).toString, "")
- append(buffer, "Location:", table.storage.locationUri.map(CatalogUtils.URIToString(_))
- .getOrElse(""), "")
- append(buffer, "Table Type:", table.tableType.name, "")
- append(buffer, "Comment:", table.comment.getOrElse(""), "")
- table.stats.foreach(s => append(buffer, "Statistics:", s.simpleString, ""))
-
- append(buffer, "Table Parameters:", "", "")
- table.properties.foreach { case (key, value) =>
- append(buffer, s" $key", value, "")
- }
-
- describeStorageInfo(table, buffer)
-
- if (table.tableType == CatalogTableType.VIEW) describeViewInfo(table, buffer)
-
- if (DDLUtils.isDatasourceTable(table) && table.tracksPartitionsInCatalog) {
- append(buffer, "Partition Provider:", "Catalog", "")
- }
- }
-
- private def describeStorageInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
- append(buffer, "", "", "")
- append(buffer, "# Storage Information", "", "")
- metadata.storage.serde.foreach(serdeLib => append(buffer, "SerDe Library:", serdeLib, ""))
- metadata.storage.inputFormat.foreach(format => append(buffer, "InputFormat:", format, ""))
- metadata.storage.outputFormat.foreach(format => append(buffer, "OutputFormat:", format, ""))
- append(buffer, "Compressed:", if (metadata.storage.compressed) "Yes" else "No", "")
- describeBucketingInfo(metadata, buffer)
-
- append(buffer, "Storage Desc Parameters:", "", "")
- val maskedProperties = CatalogUtils.maskCredentials(metadata.storage.properties)
- maskedProperties.foreach { case (key, value) =>
- append(buffer, s" $key", value, "")
- }
- }
-
- private def describeViewInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
- append(buffer, "", "", "")
- append(buffer, "# View Information", "", "")
- append(buffer, "View Text:", metadata.viewText.getOrElse(""), "")
- append(buffer, "View Default Database:", metadata.viewDefaultDatabase.getOrElse(""), "")
- append(buffer, "View Query Output Columns:",
- metadata.viewQueryColumnNames.mkString("[", ", ", "]"), "")
- }
-
- private def describeBucketingInfo(metadata: CatalogTable, buffer: ArrayBuffer[Row]): Unit = {
- metadata.bucketSpec match {
- case Some(BucketSpec(numBuckets, bucketColumnNames, sortColumnNames)) =>
- append(buffer, "Num Buckets:", numBuckets.toString, "")
- append(buffer, "Bucket Columns:", bucketColumnNames.mkString("[", ", ", "]"), "")
- append(buffer, "Sort Columns:", sortColumnNames.mkString("[", ", ", "]"), "")
-
- case _ =>
+ table.toLinkedHashMap.filterKeys(!excludedTableInfo.contains(_)).foreach {
+ s => append(buffer, s._1, s._2, "")
}
}
@@ -637,21 +578,7 @@ case class DescribeTableCommand(
}
DDLUtils.verifyPartitionProviderIsHive(spark, metadata, "DESC PARTITION")
val partition = catalog.getPartition(table, partitionSpec)
- if (isExtended) {
- describeExtendedDetailedPartitionInfo(table, metadata, partition, result)
- } else if (isFormatted) {
- describeFormattedDetailedPartitionInfo(table, metadata, partition, result)
- describeStorageInfo(metadata, result)
- }
- }
-
- private def describeExtendedDetailedPartitionInfo(
- tableIdentifier: TableIdentifier,
- table: CatalogTable,
- partition: CatalogTablePartition,
- buffer: ArrayBuffer[Row]): Unit = {
- append(buffer, "", "", "")
- append(buffer, "Detailed Partition Information " + partition.toString, "", "")
+ if (isExtended) describeFormattedDetailedPartitionInfo(table, metadata, partition, result)
}
private def describeFormattedDetailedPartitionInfo(
@@ -661,18 +588,21 @@ case class DescribeTableCommand(
buffer: ArrayBuffer[Row]): Unit = {
append(buffer, "", "", "")
append(buffer, "# Detailed Partition Information", "", "")
- append(buffer, "Partition Value:", s"[${partition.spec.values.mkString(", ")}]", "")
- append(buffer, "Database:", table.database, "")
- append(buffer, "Table:", tableIdentifier.table, "")
- append(buffer, "Location:", partition.storage.locationUri.map(CatalogUtils.URIToString(_))
- .getOrElse(""), "")
- append(buffer, "Partition Parameters:", "", "")
- partition.parameters.foreach { case (key, value) =>
- append(buffer, s" $key", value, "")
+ append(buffer, "Database", table.database, "")
+ append(buffer, "Table", tableIdentifier.table, "")
+ partition.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, ""))
+ append(buffer, "", "", "")
+ append(buffer, "# Storage Information", "", "")
+ table.bucketSpec match {
+ case Some(spec) =>
+ spec.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, ""))
+ case _ =>
}
+ table.storage.toLinkedHashMap.foreach(s => append(buffer, s._1, s._2, ""))
}
private def describeSchema(schema: StructType, buffer: ArrayBuffer[Row]): Unit = {
+ append(buffer, s"# ${output.head.name}", output(1).name, output(2).name)
schema.foreach { column =>
append(buffer, column.name, column.dataType.simpleString, column.getComment().orNull)
}
@@ -728,7 +658,7 @@ case class ShowTablesCommand(
val tableName = tableIdent.table
val isTemp = catalog.isTemporaryTable(tableIdent)
if (isExtended) {
- val information = catalog.getTempViewOrPermanentTableMetadata(tableIdent).toString
+ val information = catalog.getTempViewOrPermanentTableMetadata(tableIdent).simpleString
Row(database, tableName, isTemp, s"$information\n")
} else {
Row(database, tableName, isTemp)
@@ -745,7 +675,7 @@ case class ShowTablesCommand(
val database = table.database.getOrElse("")
val tableName = table.table
val isTemp = catalog.isTemporaryTable(table)
- val information = partition.toString
+ val information = partition.simpleString
Seq(Row(database, tableName, isTemp, s"$information\n"))
}
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
index c350d8bcbae97..e5c7c383d708c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
@@ -21,12 +21,10 @@ import java.util.concurrent.Callable
import scala.collection.mutable.ArrayBuffer
-import org.apache.hadoop.fs.Path
-
import org.apache.spark.internal.Logging
import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.{CatalystConf, CatalystTypeConverters, InternalRow, QualifiedTableName, TableIdentifier}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow, QualifiedTableName, TableIdentifier}
import org.apache.spark.sql.catalyst.CatalystTypeConverters.convertToScala
import org.apache.spark.sql.catalyst.analysis._
import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, CatalogUtils}
@@ -38,7 +36,7 @@ import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, UnknownPa
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.execution.{RowDataSourceScanExec, SparkPlan}
import org.apache.spark.sql.execution.command._
-import org.apache.spark.sql.internal.StaticSQLConf
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources._
import org.apache.spark.sql.types._
import org.apache.spark.unsafe.types.UTF8String
@@ -50,7 +48,7 @@ import org.apache.spark.unsafe.types.UTF8String
* Note that, this rule must be run after `PreprocessTableCreation` and
* `PreprocessTableInsertion`.
*/
-case class DataSourceAnalysis(conf: CatalystConf) extends Rule[LogicalPlan] {
+case class DataSourceAnalysis(conf: SQLConf) extends Rule[LogicalPlan] {
def resolver: Resolver = conf.resolver
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
index 04a764bee2ef2..3b14b794fd08c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/LogicalRelation.scala
@@ -16,11 +16,11 @@
*/
package org.apache.spark.sql.execution.datasources
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
import org.apache.spark.sql.catalyst.catalog.CatalogTable
import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeReference}
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, LogicalPlan, Statistics}
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.sources.BaseRelation
import org.apache.spark.util.Utils
@@ -73,7 +73,7 @@ case class LogicalRelation(
// expId can be different but the relation is still the same.
override lazy val cleanArgs: Seq[Any] = Seq(relation)
- @transient override def computeStats(conf: CatalystConf): Statistics = {
+ @transient override def computeStats(conf: SQLConf): Statistics = {
catalogTable.flatMap(_.stats.map(_.toPlanStats(output))).getOrElse(
Statistics(sizeInBytes = relation.sizeInBytes))
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
index 062aa5c8ea624..2f3a2c62b912c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetFileFormat.scala
@@ -125,6 +125,10 @@ class ParquetFileFormat
SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key,
sparkSession.sessionState.conf.writeLegacyParquetFormat.toString)
+ conf.set(
+ SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key,
+ sparkSession.sessionState.conf.isParquetINT64AsTimestampMillis.toString)
+
// Sets compression scheme
conf.set(ParquetOutputFormat.COMPRESSION, parquetOptions.compressionCodecClassName)
@@ -300,6 +304,9 @@ class ParquetFileFormat
hadoopConf.setBoolean(
SQLConf.PARQUET_INT96_AS_TIMESTAMP.key,
sparkSession.sessionState.conf.isParquetINT96AsTimestamp)
+ hadoopConf.setBoolean(
+ SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key,
+ sparkSession.sessionState.conf.isParquetINT64AsTimestampMillis)
// Try to push down filters when filter push-down is enabled.
val pushed =
@@ -410,7 +417,8 @@ object ParquetFileFormat extends Logging {
val converter = new ParquetSchemaConverter(
sparkSession.sessionState.conf.isParquetBinaryAsString,
sparkSession.sessionState.conf.isParquetBinaryAsString,
- sparkSession.sessionState.conf.writeLegacyParquetFormat)
+ sparkSession.sessionState.conf.writeLegacyParquetFormat,
+ sparkSession.sessionState.conf.isParquetINT64AsTimestampMillis)
converter.convert(schema)
}
@@ -510,6 +518,7 @@ object ParquetFileFormat extends Logging {
sparkSession: SparkSession): Option[StructType] = {
val assumeBinaryIsString = sparkSession.sessionState.conf.isParquetBinaryAsString
val assumeInt96IsTimestamp = sparkSession.sessionState.conf.isParquetINT96AsTimestamp
+ val writeTimestampInMillis = sparkSession.sessionState.conf.isParquetINT64AsTimestampMillis
val writeLegacyParquetFormat = sparkSession.sessionState.conf.writeLegacyParquetFormat
val serializedConf = new SerializableConfiguration(sparkSession.sessionState.newHadoopConf())
@@ -554,7 +563,8 @@ object ParquetFileFormat extends Logging {
new ParquetSchemaConverter(
assumeBinaryIsString = assumeBinaryIsString,
assumeInt96IsTimestamp = assumeInt96IsTimestamp,
- writeLegacyParquetFormat = writeLegacyParquetFormat)
+ writeLegacyParquetFormat = writeLegacyParquetFormat,
+ writeTimestampInMillis = writeTimestampInMillis)
if (footers.isEmpty) {
Iterator.empty
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala
index 33dcf2f3fd167..32e6c60cd9766 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRowConverter.scala
@@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer
import org.apache.parquet.column.Dictionary
import org.apache.parquet.io.api.{Binary, Converter, GroupConverter, PrimitiveConverter}
-import org.apache.parquet.schema.{GroupType, MessageType, Type}
+import org.apache.parquet.schema.{GroupType, MessageType, OriginalType, Type}
import org.apache.parquet.schema.OriginalType.{INT_32, LIST, UTF8}
import org.apache.parquet.schema.PrimitiveType.PrimitiveTypeName.{BINARY, DOUBLE, FIXED_LEN_BYTE_ARRAY, INT32, INT64}
@@ -252,6 +252,13 @@ private[parquet] class ParquetRowConverter(
case StringType =>
new ParquetStringConverter(updater)
+ case TimestampType if parquetType.getOriginalType == OriginalType.TIMESTAMP_MILLIS =>
+ new ParquetPrimitiveConverter(updater) {
+ override def addLong(value: Long): Unit = {
+ updater.setLong(DateTimeUtils.fromMillis(value))
+ }
+ }
+
case TimestampType =>
// TODO Implements `TIMESTAMP_MICROS` once parquet-mr has that.
new ParquetPrimitiveConverter(updater) {
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala
index 66d4027edf9f1..0b805e4362883 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaConverter.scala
@@ -51,22 +51,29 @@ import org.apache.spark.sql.types._
* and prior versions when converting a Catalyst [[StructType]] to a Parquet [[MessageType]].
* When set to false, use standard format defined in parquet-format spec. This argument only
* affects Parquet write path.
+ * @param writeTimestampInMillis Whether to write timestamp values as INT64 annotated by logical
+ * type TIMESTAMP_MILLIS.
+ *
*/
private[parquet] class ParquetSchemaConverter(
assumeBinaryIsString: Boolean = SQLConf.PARQUET_BINARY_AS_STRING.defaultValue.get,
assumeInt96IsTimestamp: Boolean = SQLConf.PARQUET_INT96_AS_TIMESTAMP.defaultValue.get,
- writeLegacyParquetFormat: Boolean = SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get) {
+ writeLegacyParquetFormat: Boolean = SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get,
+ writeTimestampInMillis: Boolean = SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.defaultValue.get) {
def this(conf: SQLConf) = this(
assumeBinaryIsString = conf.isParquetBinaryAsString,
assumeInt96IsTimestamp = conf.isParquetINT96AsTimestamp,
- writeLegacyParquetFormat = conf.writeLegacyParquetFormat)
+ writeLegacyParquetFormat = conf.writeLegacyParquetFormat,
+ writeTimestampInMillis = conf.isParquetINT64AsTimestampMillis)
def this(conf: Configuration) = this(
assumeBinaryIsString = conf.get(SQLConf.PARQUET_BINARY_AS_STRING.key).toBoolean,
assumeInt96IsTimestamp = conf.get(SQLConf.PARQUET_INT96_AS_TIMESTAMP.key).toBoolean,
writeLegacyParquetFormat = conf.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key,
- SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get.toString).toBoolean)
+ SQLConf.PARQUET_WRITE_LEGACY_FORMAT.defaultValue.get.toString).toBoolean,
+ writeTimestampInMillis = conf.get(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key).toBoolean)
+
/**
* Converts Parquet [[MessageType]] `parquetSchema` to a Spark SQL [[StructType]].
@@ -158,7 +165,7 @@ private[parquet] class ParquetSchemaConverter(
case INT_64 | null => LongType
case DECIMAL => makeDecimalType(Decimal.MAX_LONG_DIGITS)
case UINT_64 => typeNotSupported()
- case TIMESTAMP_MILLIS => typeNotImplemented()
+ case TIMESTAMP_MILLIS => TimestampType
case _ => illegalType()
}
@@ -370,10 +377,16 @@ private[parquet] class ParquetSchemaConverter(
// we may resort to microsecond precision in the future.
//
// For Parquet, we plan to write all `TimestampType` value as `TIMESTAMP_MICROS`, but it's
- // currently not implemented yet because parquet-mr 1.7.0 (the version we're currently using)
- // hasn't implemented `TIMESTAMP_MICROS` yet.
+ // currently not implemented yet because parquet-mr 1.8.1 (the version we're currently using)
+ // hasn't implemented `TIMESTAMP_MICROS` yet, however it supports TIMESTAMP_MILLIS. We will
+ // encode timestamp values as TIMESTAMP_MILLIS annotating INT64 if
+ // 'spark.sql.parquet.int64AsTimestampMillis' is set.
//
// TODO Converts `TIMESTAMP_MICROS` once parquet-mr implements that.
+
+ case TimestampType if writeTimestampInMillis =>
+ Types.primitive(INT64, repetition).as(TIMESTAMP_MILLIS).named(field.name)
+
case TimestampType =>
Types.primitive(INT96, repetition).named(field.name)
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala
index a31d2b9c37e9d..38b0e33937f3c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetWriteSupport.scala
@@ -66,6 +66,9 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
// Whether to write data in legacy Parquet format compatible with Spark 1.4 and prior versions
private var writeLegacyParquetFormat: Boolean = _
+ // Whether to write timestamp value with milliseconds precision.
+ private var writeTimestampInMillis: Boolean = _
+
// Reusable byte array used to write timestamps as Parquet INT96 values
private val timestampBuffer = new Array[Byte](12)
@@ -80,6 +83,13 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
assert(configuration.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key) != null)
configuration.get(SQLConf.PARQUET_WRITE_LEGACY_FORMAT.key).toBoolean
}
+
+ this.writeTimestampInMillis = {
+ assert(configuration.get(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key) != null)
+ configuration.get(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key).toBoolean
+ }
+
+
this.rootFieldWriters = schema.map(_.dataType).map(makeWriter)
val messageType = new ParquetSchemaConverter(configuration).convert(schema)
@@ -153,6 +163,11 @@ private[parquet] class ParquetWriteSupport extends WriteSupport[InternalRow] wit
recordConsumer.addBinary(
Binary.fromReusedByteArray(row.getUTF8String(ordinal).getBytes))
+ case TimestampType if writeTimestampInMillis =>
+ (row: SpecializedGetters, ordinal: Int) =>
+ val millis = DateTimeUtils.toMillis(row.getLong(ordinal))
+ recordConsumer.addLong(millis)
+
case TimestampType =>
(row: SpecializedGetters, ordinal: Int) => {
// TODO Writes `TimestampType` values as `TIMESTAMP_MICROS` once parquet-mr implements it
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
index 6d34d51d31c1e..971ce5afb1778 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/streaming/memory.scala
@@ -25,11 +25,11 @@ import scala.util.control.NonFatal
import org.apache.spark.internal.Logging
import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.CatalystConf
import org.apache.spark.sql.catalyst.encoders.encoderFor
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.plans.logical.{LeafNode, Statistics}
import org.apache.spark.sql.catalyst.streaming.InternalOutputModes._
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.streaming.OutputMode
import org.apache.spark.sql.types.StructType
import org.apache.spark.util.Utils
@@ -230,6 +230,6 @@ case class MemoryPlan(sink: MemorySink, output: Seq[Attribute]) extends LeafNode
private val sizePerRow = sink.schema.toAttributes.map(_.dataType.defaultSize).sum
- override def computeStats(conf: CatalystConf): Statistics =
+ override def computeStats(conf: SQLConf): Statistics =
Statistics(sizePerRow * sink.allData.size)
}
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala b/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala
index 174378304d4a5..e266ae55cc4d9 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/expressions/ReduceAggregator.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
private[sql] class ReduceAggregator[T: Encoder](func: (T, T) => T)
extends Aggregator[T, (Boolean, T), T] {
- private val encoder = implicitly[Encoder[T]]
+ @transient private val encoder = implicitly[Encoder[T]]
override def zero: (Boolean, T) = (false, null.asInstanceOf[T])
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
index 53374859f13f4..5d1c35aba529a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala
@@ -19,8 +19,6 @@ package org.apache.spark.sql.internal
import scala.reflect.runtime.universe.TypeTag
-import org.apache.hadoop.fs.Path
-
import org.apache.spark.annotation.Experimental
import org.apache.spark.sql._
import org.apache.spark.sql.catalog.{Catalog, Column, Database, Function, Table}
@@ -143,11 +141,12 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Returns a list of columns for the given table in the current database.
+ * Returns a list of columns for the given table temporary view.
*/
@throws[AnalysisException]("table does not exist")
override def listColumns(tableName: String): Dataset[Column] = {
- listColumns(TableIdentifier(tableName, None))
+ val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
+ listColumns(tableIdent)
}
/**
@@ -177,7 +176,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Get the database with the specified name. This throws an `AnalysisException` when no
+ * Gets the database with the specified name. This throws an `AnalysisException` when no
* `Database` can be found.
*/
override def getDatabase(dbName: String): Database = {
@@ -185,16 +184,16 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Get the table or view with the specified name. This table can be a temporary view or a
- * table/view in the current database. This throws an `AnalysisException` when no `Table`
- * can be found.
+ * Gets the table or view with the specified name. This table can be a temporary view or a
+ * table/view. This throws an `AnalysisException` when no `Table` can be found.
*/
override def getTable(tableName: String): Table = {
- getTable(null, tableName)
+ val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
+ getTable(tableIdent.database.orNull, tableIdent.table)
}
/**
- * Get the table or view with the specified name in the specified database. This throws an
+ * Gets the table or view with the specified name in the specified database. This throws an
* `AnalysisException` when no `Table` can be found.
*/
override def getTable(dbName: String, tableName: String): Table = {
@@ -202,16 +201,16 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Get the function with the specified name. This function can be a temporary function or a
- * function in the current database. This throws an `AnalysisException` when no `Function`
- * can be found.
+ * Gets the function with the specified name. This function can be a temporary function or a
+ * function. This throws an `AnalysisException` when no `Function` can be found.
*/
override def getFunction(functionName: String): Function = {
- getFunction(null, functionName)
+ val functionIdent = sparkSession.sessionState.sqlParser.parseFunctionIdentifier(functionName)
+ getFunction(functionIdent.database.orNull, functionIdent.funcName)
}
/**
- * Get the function with the specified name. This returns `None` when no `Function` can be
+ * Gets the function with the specified name. This returns `None` when no `Function` can be
* found.
*/
override def getFunction(dbName: String, functionName: String): Function = {
@@ -219,22 +218,23 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Check if the database with the specified name exists.
+ * Checks if the database with the specified name exists.
*/
override def databaseExists(dbName: String): Boolean = {
sessionCatalog.databaseExists(dbName)
}
/**
- * Check if the table or view with the specified name exists. This can either be a temporary
- * view or a table/view in the current database.
+ * Checks if the table or view with the specified name exists. This can either be a temporary
+ * view or a table/view.
*/
override def tableExists(tableName: String): Boolean = {
- tableExists(null, tableName)
+ val tableIdent = sparkSession.sessionState.sqlParser.parseTableIdentifier(tableName)
+ tableExists(tableIdent.database.orNull, tableIdent.table)
}
/**
- * Check if the table or view with the specified name exists in the specified database.
+ * Checks if the table or view with the specified name exists in the specified database.
*/
override def tableExists(dbName: String, tableName: String): Boolean = {
val tableIdent = TableIdentifier(tableName, Option(dbName))
@@ -242,15 +242,16 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Check if the function with the specified name exists. This can either be a temporary function
- * or a function in the current database.
+ * Checks if the function with the specified name exists. This can either be a temporary function
+ * or a function.
*/
override def functionExists(functionName: String): Boolean = {
- functionExists(null, functionName)
+ val functionIdent = sparkSession.sessionState.sqlParser.parseFunctionIdentifier(functionName)
+ functionExists(functionIdent.database.orNull, functionIdent.funcName)
}
/**
- * Check if the function with the specified name exists in the specified database.
+ * Checks if the function with the specified name exists in the specified database.
*/
override def functionExists(dbName: String, functionName: String): Boolean = {
sessionCatalog.functionExists(FunctionIdentifier(functionName, Option(dbName)))
@@ -303,7 +304,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
/**
* :: Experimental ::
* (Scala-specific)
- * Create a table from the given path based on a data source, a schema and a set of options.
+ * Creates a table from the given path based on a data source, a schema and a set of options.
* Then, returns the corresponding DataFrame.
*
* @group ddl_ops
@@ -338,7 +339,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
* Drops the local temporary view with the given view name in the catalog.
* If the view has been cached/persisted before, it's also unpersisted.
*
- * @param viewName the name of the view to be dropped.
+ * @param viewName the identifier of the temporary view to be dropped.
* @group ddl_ops
* @since 2.0.0
*/
@@ -353,7 +354,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
* Drops the global temporary view with the given view name in the catalog.
* If the view has been cached/persisted before, it's also unpersisted.
*
- * @param viewName the name of the view to be dropped.
+ * @param viewName the identifier of the global temporary view to be dropped.
* @group ddl_ops
* @since 2.1.0
*/
@@ -365,9 +366,11 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Recover all the partitions in the directory of a table and update the catalog.
+ * Recovers all the partitions in the directory of a table and update the catalog.
*
- * @param tableName the name of the table to be repaired.
+ * @param tableName is either a qualified or unqualified name that designates a table.
+ * If no database identifier is provided, it refers to a table in the
+ * current database.
* @group ddl_ops
* @since 2.1.1
*/
@@ -378,7 +381,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Returns true if the table is currently cached in-memory.
+ * Returns true if the table or view is currently cached in-memory.
*
* @group cachemgmt
* @since 2.0.0
@@ -388,7 +391,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Caches the specified table in-memory.
+ * Caches the specified table or view in-memory.
*
* @group cachemgmt
* @since 2.0.0
@@ -398,7 +401,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Removes the specified table from the in-memory cache.
+ * Removes the specified table or view from the in-memory cache.
*
* @group cachemgmt
* @since 2.0.0
@@ -408,7 +411,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Removes all cached tables from the in-memory cache.
+ * Removes all cached tables or views from the in-memory cache.
*
* @group cachemgmt
* @since 2.0.0
@@ -428,7 +431,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Refresh the cache entry for a table, if any. For Hive metastore table, the metadata
+ * Refreshes the cache entry for a table or view, if any. For Hive metastore table, the metadata
* is refreshed. For data source tables, the schema will not be inferred and refreshed.
*
* @group cachemgmt
@@ -452,7 +455,7 @@ class CatalogImpl(sparkSession: SparkSession) extends Catalog {
}
/**
- * Refresh the cache entry and the associated metadata for all dataframes (if any), that contain
+ * Refreshes the cache entry and the associated metadata for all Dataset (if any), that contain
* the given data source path.
*
* @group cachemgmt
diff --git a/sql/core/src/test/resources/sql-tests/inputs/describe.sql b/sql/core/src/test/resources/sql-tests/inputs/describe.sql
index 56f3281440d29..6de4cf0d5afa1 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/describe.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/describe.sql
@@ -1,10 +1,23 @@
-CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) COMMENT 'table_comment';
+CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet
+ PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS
+ COMMENT 'table_comment';
+
+CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t;
+
+CREATE TEMPORARY VIEW temp_Data_Source_View
+ USING org.apache.spark.sql.sources.DDLScanSource
+ OPTIONS (
+ From '1',
+ To '10',
+ Table 'test1');
+
+CREATE VIEW v AS SELECT * FROM t;
ALTER TABLE t ADD PARTITION (c='Us', d=1);
DESCRIBE t;
-DESC t;
+DESC default.t;
DESC TABLE t;
@@ -27,5 +40,39 @@ DESC t PARTITION (c='Us');
-- ParseException: PARTITION specification is incomplete
DESC t PARTITION (c='Us', d);
--- DROP TEST TABLE
+-- DESC Temp View
+
+DESC temp_v;
+
+DESC TABLE temp_v;
+
+DESC FORMATTED temp_v;
+
+DESC EXTENDED temp_v;
+
+DESC temp_Data_Source_View;
+
+-- AnalysisException DESC PARTITION is not allowed on a temporary view
+DESC temp_v PARTITION (c='Us', d=1);
+
+-- DESC Persistent View
+
+DESC v;
+
+DESC TABLE v;
+
+DESC FORMATTED v;
+
+DESC EXTENDED v;
+
+-- AnalysisException DESC PARTITION is not allowed on a view
+DESC v PARTITION (c='Us', d=1);
+
+-- DROP TEST TABLES/VIEWS
DROP TABLE t;
+
+DROP VIEW temp_v;
+
+DROP VIEW temp_Data_Source_View;
+
+DROP VIEW v;
diff --git a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql
index 2e6dcd538b7ac..d0d2df7b243d5 100644
--- a/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql
+++ b/sql/core/src/test/resources/sql-tests/inputs/table-valued-functions.sql
@@ -18,3 +18,9 @@ select * from range(1, 1, 1, 1, 1);
-- range call with null
select * from range(1, null);
+
+-- range call with a mixed-case function name
+select * from RaNgE(2);
+
+-- Explain
+EXPLAIN select * from RaNgE(2);
diff --git a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
index ba8bc936f0c79..678a3f0f0a3c6 100644
--- a/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/change-column.sql.out
@@ -15,6 +15,7 @@ DESC test_change
-- !query 1 schema
struct
-- !query 1 output
+# col_name data_type comment
a int
b string
c int
@@ -34,6 +35,7 @@ DESC test_change
-- !query 3 schema
struct
-- !query 3 output
+# col_name data_type comment
a int
b string
c int
@@ -53,6 +55,7 @@ DESC test_change
-- !query 5 schema
struct
-- !query 5 output
+# col_name data_type comment
a int
b string
c int
@@ -91,6 +94,7 @@ DESC test_change
-- !query 8 schema
struct
-- !query 8 output
+# col_name data_type comment
a int
b string
c int
@@ -125,6 +129,7 @@ DESC test_change
-- !query 12 schema
struct
-- !query 12 output
+# col_name data_type comment
a int this is column a
b string #*02?`
c int
@@ -143,6 +148,7 @@ DESC test_change
-- !query 14 schema
struct
-- !query 14 output
+# col_name data_type comment
a int this is column a
b string #*02?`
c int
@@ -162,6 +168,7 @@ DESC test_change
-- !query 16 schema
struct
-- !query 16 output
+# col_name data_type comment
a int this is column a
b string #*02?`
c int
@@ -186,6 +193,7 @@ DESC test_change
-- !query 18 schema
struct
-- !query 18 output
+# col_name data_type comment
a int this is column a
b string #*02?`
c int
@@ -229,6 +237,7 @@ DESC test_change
-- !query 23 schema
struct
-- !query 23 output
+# col_name data_type comment
a int this is column A
b string #*02?`
c int
diff --git a/sql/core/src/test/resources/sql-tests/results/describe.sql.out b/sql/core/src/test/resources/sql-tests/results/describe.sql.out
index 422d548ea8de8..de10b29f3c65b 100644
--- a/sql/core/src/test/resources/sql-tests/results/describe.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/describe.sql.out
@@ -1,9 +1,11 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 14
+-- Number of queries: 31
-- !query 0
-CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet PARTITIONED BY (c, d) COMMENT 'table_comment'
+CREATE TABLE t (a STRING, b INT, c STRING, d STRING) USING parquet
+ PARTITIONED BY (c, d) CLUSTERED BY (a) SORTED BY (b ASC) INTO 2 BUCKETS
+ COMMENT 'table_comment'
-- !query 0 schema
struct<>
-- !query 0 output
@@ -11,7 +13,7 @@ struct<>
-- !query 1
-ALTER TABLE t ADD PARTITION (c='Us', d=1)
+CREATE TEMPORARY VIEW temp_v AS SELECT * FROM t
-- !query 1 schema
struct<>
-- !query 1 output
@@ -19,187 +21,239 @@ struct<>
-- !query 2
-DESCRIBE t
+CREATE TEMPORARY VIEW temp_Data_Source_View
+ USING org.apache.spark.sql.sources.DDLScanSource
+ OPTIONS (
+ From '1',
+ To '10',
+ Table 'test1')
-- !query 2 schema
-struct
+struct<>
-- !query 2 output
-# Partition Information
+
+
+
+-- !query 3
+CREATE VIEW v AS SELECT * FROM t
+-- !query 3 schema
+struct<>
+-- !query 3 output
+
+
+
+-- !query 4
+ALTER TABLE t ADD PARTITION (c='Us', d=1)
+-- !query 4 schema
+struct<>
+-- !query 4 output
+
+
+
+-- !query 5
+DESCRIBE t
+-- !query 5 schema
+struct
+-- !query 5 output
# col_name data_type comment
a string
b int
c string
-c string
d string
+# Partition Information
+# col_name data_type comment
+c string
d string
--- !query 3
-DESC t
--- !query 3 schema
+-- !query 6
+DESC default.t
+-- !query 6 schema
struct
--- !query 3 output
-# Partition Information
+-- !query 6 output
# col_name data_type comment
a string
b int
c string
-c string
d string
+# Partition Information
+# col_name data_type comment
+c string
d string
--- !query 4
+-- !query 7
DESC TABLE t
--- !query 4 schema
+-- !query 7 schema
struct
--- !query 4 output
-# Partition Information
+-- !query 7 output
# col_name data_type comment
a string
b int
c string
-c string
d string
+# Partition Information
+# col_name data_type comment
+c string
d string
--- !query 5
+-- !query 8
DESC FORMATTED t
--- !query 5 schema
+-- !query 8 schema
struct
--- !query 5 output
-# Detailed Table Information
-# Partition Information
-# Storage Information
+-- !query 8 output
# col_name data_type comment
-Comment: table_comment
-Compressed: No
-Created:
-Database: default
-Last Access:
-Location: sql/core/spark-warehouse/t
-Owner:
-Partition Provider: Catalog
-Storage Desc Parameters:
-Table Parameters:
-Table Type: MANAGED
a string
b int
c string
+d string
+# Partition Information
+# col_name data_type comment
c string
d string
-d string
+
+# Detailed Table Information
+Database default
+Table t
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type MANAGED
+Provider parquet
+Num Buckets 2
+Bucket Columns [`a`]
+Sort Columns [`b`]
+Comment table_comment
+Location [not included in comparison]sql/core/spark-warehouse/t
+Partition Provider Catalog
--- !query 6
+-- !query 9
DESC EXTENDED t
--- !query 6 schema
+-- !query 9 schema
struct
--- !query 6 output
-# Detailed Table Information CatalogTable(
- Table: `default`.`t`
- Created:
- Last Access:
- Type: MANAGED
- Schema: [StructField(a,StringType,true), StructField(b,IntegerType,true), StructField(c,StringType,true), StructField(d,StringType,true)]
- Provider: parquet
- Partition Columns: [`c`, `d`]
- Comment: table_comment
- Storage(Location: sql/core/spark-warehouse/t)
- Partition Provider: Catalog)
-# Partition Information
+-- !query 9 output
# col_name data_type comment
a string
b int
c string
+d string
+# Partition Information
+# col_name data_type comment
c string
d string
-d string
+
+# Detailed Table Information
+Database default
+Table t
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type MANAGED
+Provider parquet
+Num Buckets 2
+Bucket Columns [`a`]
+Sort Columns [`b`]
+Comment table_comment
+Location [not included in comparison]sql/core/spark-warehouse/t
+Partition Provider Catalog
--- !query 7
+-- !query 10
DESC t PARTITION (c='Us', d=1)
--- !query 7 schema
+-- !query 10 schema
struct
--- !query 7 output
-# Partition Information
+-- !query 10 output
# col_name data_type comment
a string
b int
c string
-c string
d string
+# Partition Information
+# col_name data_type comment
+c string
d string
--- !query 8
+-- !query 11
DESC EXTENDED t PARTITION (c='Us', d=1)
--- !query 8 schema
+-- !query 11 schema
struct
--- !query 8 output
-# Partition Information
+-- !query 11 output
# col_name data_type comment
-Detailed Partition Information CatalogPartition(
- Partition Values: [c=Us, d=1]
- Storage(Location: sql/core/spark-warehouse/t/c=Us/d=1)
- Partition Parameters:{})
a string
b int
c string
+d string
+# Partition Information
+# col_name data_type comment
c string
d string
-d string
+
+# Detailed Partition Information
+Database default
+Table t
+Partition Values [c=Us, d=1]
+Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1
+
+# Storage Information
+Num Buckets 2
+Bucket Columns [`a`]
+Sort Columns [`b`]
+Location [not included in comparison]sql/core/spark-warehouse/t
--- !query 9
+-- !query 12
DESC FORMATTED t PARTITION (c='Us', d=1)
--- !query 9 schema
+-- !query 12 schema
struct
--- !query 9 output
-# Detailed Partition Information
-# Partition Information
-# Storage Information
+-- !query 12 output
# col_name data_type comment
-Compressed: No
-Database: default
-Location: sql/core/spark-warehouse/t/c=Us/d=1
-Partition Parameters:
-Partition Value: [Us, 1]
-Storage Desc Parameters:
-Table: t
a string
b int
c string
+d string
+# Partition Information
+# col_name data_type comment
c string
d string
-d string
+
+# Detailed Partition Information
+Database default
+Table t
+Partition Values [c=Us, d=1]
+Location [not included in comparison]sql/core/spark-warehouse/t/c=Us/d=1
+
+# Storage Information
+Num Buckets 2
+Bucket Columns [`a`]
+Sort Columns [`b`]
+Location [not included in comparison]sql/core/spark-warehouse/t
--- !query 10
+-- !query 13
DESC t PARTITION (c='Us', d=2)
--- !query 10 schema
+-- !query 13 schema
struct<>
--- !query 10 output
+-- !query 13 output
org.apache.spark.sql.catalyst.analysis.NoSuchPartitionException
Partition not found in table 't' database 'default':
c -> Us
d -> 2;
--- !query 11
+-- !query 14
DESC t PARTITION (c='Us')
--- !query 11 schema
+-- !query 14 schema
struct<>
--- !query 11 output
+-- !query 14 output
org.apache.spark.sql.AnalysisException
Partition spec is invalid. The spec (c) must match the partition spec (c, d) defined in table '`default`.`t`';
--- !query 12
+-- !query 15
DESC t PARTITION (c='Us', d)
--- !query 12 schema
+-- !query 15 schema
struct<>
--- !query 12 output
+-- !query 15 output
org.apache.spark.sql.catalyst.parser.ParseException
PARTITION specification is incomplete: `d`(line 1, pos 0)
@@ -209,9 +263,193 @@ DESC t PARTITION (c='Us', d)
^^^
--- !query 13
+-- !query 16
+DESC temp_v
+-- !query 16 schema
+struct
+-- !query 16 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+
+-- !query 17
+DESC TABLE temp_v
+-- !query 17 schema
+struct
+-- !query 17 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+
+-- !query 18
+DESC FORMATTED temp_v
+-- !query 18 schema
+struct
+-- !query 18 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+
+-- !query 19
+DESC EXTENDED temp_v
+-- !query 19 schema
+struct
+-- !query 19 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+
+-- !query 20
+DESC temp_Data_Source_View
+-- !query 20 schema
+struct
+-- !query 20 output
+# col_name data_type comment
+intType int test comment test1
+stringType string
+dateType date
+timestampType timestamp
+doubleType double
+bigintType bigint
+tinyintType tinyint
+decimalType decimal(10,0)
+fixedDecimalType decimal(5,1)
+binaryType binary
+booleanType boolean
+smallIntType smallint
+floatType float
+mapType map
+arrayType array
+structType struct
+
+
+-- !query 21
+DESC temp_v PARTITION (c='Us', d=1)
+-- !query 21 schema
+struct<>
+-- !query 21 output
+org.apache.spark.sql.AnalysisException
+DESC PARTITION is not allowed on a temporary view: temp_v;
+
+
+-- !query 22
+DESC v
+-- !query 22 schema
+struct
+-- !query 22 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+
+-- !query 23
+DESC TABLE v
+-- !query 23 schema
+struct
+-- !query 23 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+
+-- !query 24
+DESC FORMATTED v
+-- !query 24 schema
+struct
+-- !query 24 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+# Detailed Table Information
+Database default
+Table v
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type VIEW
+View Text SELECT * FROM t
+View Default Database default
+View Query Output Columns [a, b, c, d]
+Properties [view.query.out.col.3=d, view.query.out.col.0=a, view.query.out.numCols=4, view.default.database=default, view.query.out.col.1=b, view.query.out.col.2=c]
+
+
+-- !query 25
+DESC EXTENDED v
+-- !query 25 schema
+struct
+-- !query 25 output
+# col_name data_type comment
+a string
+b int
+c string
+d string
+
+# Detailed Table Information
+Database default
+Table v
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type VIEW
+View Text SELECT * FROM t
+View Default Database default
+View Query Output Columns [a, b, c, d]
+Properties [view.query.out.col.3=d, view.query.out.col.0=a, view.query.out.numCols=4, view.default.database=default, view.query.out.col.1=b, view.query.out.col.2=c]
+
+
+-- !query 26
+DESC v PARTITION (c='Us', d=1)
+-- !query 26 schema
+struct<>
+-- !query 26 output
+org.apache.spark.sql.AnalysisException
+DESC PARTITION is not allowed on a view: v;
+
+
+-- !query 27
DROP TABLE t
--- !query 13 schema
+-- !query 27 schema
struct<>
--- !query 13 output
+-- !query 27 output
+
+
+
+-- !query 28
+DROP VIEW temp_v
+-- !query 28 schema
+struct<>
+-- !query 28 output
+
+
+
+-- !query 29
+DROP VIEW temp_Data_Source_View
+-- !query 29 schema
+struct<>
+-- !query 29 output
+
+
+
+-- !query 30
+DROP VIEW v
+-- !query 30 schema
+struct<>
+-- !query 30 output
diff --git a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
index 6d62e6092147b..8f2a54f7c24e2 100644
--- a/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/show-tables.sql.out
@@ -118,33 +118,40 @@ SHOW TABLE EXTENDED LIKE 'show_t*'
-- !query 12 schema
struct
-- !query 12 output
-show_t3 true CatalogTable(
- Table: `show_t3`
- Created:
- Last Access:
- Type: VIEW
- Schema: [StructField(e,IntegerType,true)]
- Storage())
-
-showdb show_t1 false CatalogTable(
- Table: `showdb`.`show_t1`
- Created:
- Last Access:
- Type: MANAGED
- Schema: [StructField(a,StringType,true), StructField(b,IntegerType,true), StructField(c,StringType,true), StructField(d,StringType,true)]
- Provider: parquet
- Partition Columns: [`c`, `d`]
- Storage(Location: sql/core/spark-warehouse/showdb.db/show_t1)
- Partition Provider: Catalog)
-
-showdb show_t2 false CatalogTable(
- Table: `showdb`.`show_t2`
- Created:
- Last Access:
- Type: MANAGED
- Schema: [StructField(b,StringType,true), StructField(d,IntegerType,true)]
- Provider: parquet
- Storage(Location: sql/core/spark-warehouse/showdb.db/show_t2))
+show_t3 true Table: show_t3
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type: VIEW
+Schema: root
+ |-- e: integer (nullable = true)
+
+
+showdb show_t1 false Database: showdb
+Table: show_t1
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type: MANAGED
+Provider: parquet
+Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1
+Partition Provider: Catalog
+Partition Columns: [`c`, `d`]
+Schema: root
+ |-- a: string (nullable = true)
+ |-- b: integer (nullable = true)
+ |-- c: string (nullable = true)
+ |-- d: string (nullable = true)
+
+
+showdb show_t2 false Database: showdb
+Table: show_t2
+Created [not included in comparison]
+Last Access [not included in comparison]
+Type: MANAGED
+Provider: parquet
+Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t2
+Schema: root
+ |-- b: string (nullable = true)
+ |-- d: integer (nullable = true)
-- !query 13
@@ -166,10 +173,8 @@ SHOW TABLE EXTENDED LIKE 'show_t1' PARTITION(c='Us', d=1)
-- !query 14 schema
struct
-- !query 14 output
-showdb show_t1 false CatalogPartition(
- Partition Values: [c=Us, d=1]
- Storage(Location: sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1)
- Partition Parameters:{})
+showdb show_t1 false Partition Values: [c=Us, d=1]
+Location [not included in comparison]sql/core/spark-warehouse/showdb.db/show_t1/c=Us/d=1
-- !query 15
diff --git a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out
index d769bcef0aca7..acd4ecf14617e 100644
--- a/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out
+++ b/sql/core/src/test/resources/sql-tests/results/table-valued-functions.sql.out
@@ -1,5 +1,5 @@
-- Automatically generated by SQLQueryTestSuite
--- Number of queries: 7
+-- Number of queries: 9
-- !query 0
@@ -85,3 +85,21 @@ struct<>
-- !query 6 output
java.lang.IllegalArgumentException
Invalid arguments for resolved function: 1, null
+
+
+-- !query 7
+select * from RaNgE(2)
+-- !query 7 schema
+struct
+-- !query 7 output
+0
+1
+
+
+-- !query 8
+EXPLAIN select * from RaNgE(2)
+-- !query 8 schema
+struct
+-- !query 8 output
+== Physical Plan ==
+*Range (0, 2, step=1, splits=None)
diff --git a/sql/core/src/test/resources/test-data/timemillis-in-i64.parquet b/sql/core/src/test/resources/test-data/timemillis-in-i64.parquet
new file mode 100644
index 0000000000000..d3c39e2c26eec
Binary files /dev/null and b/sql/core/src/test/resources/test-data/timemillis-in-i64.parquet differ
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
index 68e071a1a694f..5b5cd28ad0c99 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/DatasetSuite.scala
@@ -142,6 +142,15 @@ class DatasetSuite extends QueryTest with SharedSQLContext {
assert(ds.take(2) === Array(ClassData("a", 1), ClassData("b", 2)))
}
+ test("as seq of case class - reorder fields by name") {
+ val df = spark.range(3).select(array(struct($"id".cast("int").as("b"), lit("a").as("a"))))
+ val ds = df.as[Seq[ClassData]]
+ assert(ds.collect() === Array(
+ Seq(ClassData("a", 0)),
+ Seq(ClassData("a", 1)),
+ Seq(ClassData("a", 2))))
+ }
+
test("map") {
val ds = Seq(("a", 1), ("b", 2), ("c", 3)).toDS()
checkDataset(
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
index 4092862c430b1..4b69baffab620 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQueryTestSuite.scala
@@ -26,6 +26,7 @@ import org.apache.spark.sql.catalyst.planning.PhysicalOperation
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules.RuleExecutor
import org.apache.spark.sql.catalyst.util.{fileToString, stringToFile}
+import org.apache.spark.sql.execution.command.DescribeTableCommand
import org.apache.spark.sql.test.SharedSQLContext
import org.apache.spark.sql.types.StructType
@@ -165,8 +166,8 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
s"-- Number of queries: ${outputs.size}\n\n\n" +
outputs.zipWithIndex.map{case (qr, i) => qr.toString(i)}.mkString("\n\n\n") + "\n"
}
- val resultFile = new File(testCase.resultFile);
- val parent = resultFile.getParentFile();
+ val resultFile = new File(testCase.resultFile)
+ val parent = resultFile.getParentFile
if (!parent.exists()) {
assert(parent.mkdirs(), "Could not create directory: " + parent)
}
@@ -212,23 +213,25 @@ class SQLQueryTestSuite extends QueryTest with SharedSQLContext {
/** Executes a query and returns the result as (schema of the output, normalized output). */
private def getNormalizedResult(session: SparkSession, sql: String): (StructType, Seq[String]) = {
// Returns true if the plan is supposed to be sorted.
- def isSorted(plan: LogicalPlan): Boolean = plan match {
+ def needSort(plan: LogicalPlan): Boolean = plan match {
case _: Join | _: Aggregate | _: Generate | _: Sample | _: Distinct => false
+ case _: DescribeTableCommand => true
case PhysicalOperation(_, _, Sort(_, true, _)) => true
- case _ => plan.children.iterator.exists(isSorted)
+ case _ => plan.children.iterator.exists(needSort)
}
try {
val df = session.sql(sql)
val schema = df.schema
+ val notIncludedMsg = "[not included in comparison]"
// Get answer, but also get rid of the #1234 expression ids that show up in explain plans
val answer = df.queryExecution.hiveResultString().map(_.replaceAll("#\\d+", "#x")
- .replaceAll("Location:.*/sql/core/", "Location: sql/core/")
- .replaceAll("Created: .*", "Created: ")
- .replaceAll("Last Access: .*", "Last Access: "))
+ .replaceAll("Location.*/sql/core/", s"Location ${notIncludedMsg}sql/core/")
+ .replaceAll("Created.*", s"Created $notIncludedMsg")
+ .replaceAll("Last Access.*", s"Last Access $notIncludedMsg"))
// If the output is not pre-sorted, sort it.
- if (isSorted(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
+ if (needSort(df.queryExecution.analyzed)) (schema, answer) else (schema, answer.sorted)
} catch {
case a: AnalysisException =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
index a4d012cd76115..908b955abbf07 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/SparkSqlParserSuite.scala
@@ -224,13 +224,13 @@ class SparkSqlParserSuite extends PlanTest {
test("SPARK-17328 Fix NPE with EXPLAIN DESCRIBE TABLE") {
assertEqual("describe table t",
DescribeTableCommand(
- TableIdentifier("t"), Map.empty, isExtended = false, isFormatted = false))
+ TableIdentifier("t"), Map.empty, isExtended = false))
assertEqual("describe table extended t",
DescribeTableCommand(
- TableIdentifier("t"), Map.empty, isExtended = true, isFormatted = false))
+ TableIdentifier("t"), Map.empty, isExtended = true))
assertEqual("describe table formatted t",
DescribeTableCommand(
- TableIdentifier("t"), Map.empty, isExtended = false, isFormatted = true))
+ TableIdentifier("t"), Map.empty, isExtended = true))
intercept("explain describe tables x", "Unsupported SQL statement")
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
index 648b1798c66e0..9ebf2dd839a79 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/command/DDLSuite.scala
@@ -69,18 +69,6 @@ class InMemoryCatalogedDDLSuite extends DDLSuite with SharedSQLContext with Befo
tracksPartitionsInCatalog = true)
}
- test("desc table for parquet data source table using in-memory catalog") {
- val tabName = "tab1"
- withTable(tabName) {
- sql(s"CREATE TABLE $tabName(a int comment 'test') USING parquet ")
-
- checkAnswer(
- sql(s"DESC $tabName").select("col_name", "data_type", "comment"),
- Row("a", "int", "test")
- )
- }
- }
-
test("alter table: set location (datasource table)") {
testSetLocation(isDatasourceTable = true)
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
index dbdcd230a4de9..57a0af1dda971 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetIOSuite.scala
@@ -107,11 +107,13 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
| required binary g(ENUM);
| required binary h(DECIMAL(32,0));
| required fixed_len_byte_array(32) i(DECIMAL(32,0));
+ | required int64 j(TIMESTAMP_MILLIS);
|}
""".stripMargin)
val expectedSparkTypes = Seq(ByteType, ShortType, DateType, DecimalType(1, 0),
- DecimalType(10, 0), StringType, StringType, DecimalType(32, 0), DecimalType(32, 0))
+ DecimalType(10, 0), StringType, StringType, DecimalType(32, 0), DecimalType(32, 0),
+ TimestampType)
withTempPath { location =>
val path = new Path(location.getCanonicalPath)
@@ -607,6 +609,18 @@ class ParquetIOSuite extends QueryTest with ParquetTest with SharedSQLContext {
}
}
+ test("read dictionary and plain encoded timestamp_millis written as INT64") {
+ ("true" :: "false" :: Nil).foreach { vectorized =>
+ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) {
+ checkAnswer(
+ // timestamp column in this file is encoded using combination of plain
+ // and dictionary encodings.
+ readResourceParquetFile("test-data/timemillis-in-i64.parquet"),
+ (1 to 3).map(i => Row(new java.sql.Timestamp(10))))
+ }
+ }
+ }
+
test("SPARK-12589 copy() on rows returned from reader works for strings") {
withTempPath { dir =>
val data = (1, "abc") ::(2, "helloabcde") :: Nil
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
index 200e356c72fd7..c36609586c807 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetQuerySuite.scala
@@ -18,6 +18,7 @@
package org.apache.spark.sql.execution.datasources.parquet
import java.io.File
+import java.sql.Timestamp
import org.apache.hadoop.fs.{FileSystem, Path}
import org.apache.parquet.hadoop.ParquetOutputFormat
@@ -162,6 +163,78 @@ class ParquetQuerySuite extends QueryTest with ParquetTest with SharedSQLContext
}
}
+ test("SPARK-10634 timestamp written and read as INT64 - TIMESTAMP_MILLIS") {
+ val data = (1 to 10).map(i => Row(i, new java.sql.Timestamp(i)))
+ val schema = StructType(List(StructField("d", IntegerType, false),
+ StructField("time", TimestampType, false)).toArray)
+ withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") {
+ withTempPath { file =>
+ val df = spark.createDataFrame(sparkContext.parallelize(data), schema)
+ df.write.parquet(file.getCanonicalPath)
+ ("true" :: "false" :: Nil).foreach { vectorized =>
+ withSQLConf(SQLConf.PARQUET_VECTORIZED_READER_ENABLED.key -> vectorized) {
+ val df2 = spark.read.parquet(file.getCanonicalPath)
+ checkAnswer(df2, df.collect().toSeq)
+ }
+ }
+ }
+ }
+ }
+
+ test("SPARK-10634 timestamp written and read as INT64 - truncation") {
+ withTable("ts") {
+ sql("create table ts (c1 int, c2 timestamp) using parquet")
+ sql("insert into ts values (1, '2016-01-01 10:11:12.123456')")
+ sql("insert into ts values (2, null)")
+ sql("insert into ts values (3, '1965-01-01 10:11:12.123456')")
+ checkAnswer(
+ sql("select * from ts"),
+ Seq(
+ Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123456")),
+ Row(2, null),
+ Row(3, Timestamp.valueOf("1965-01-01 10:11:12.123456"))))
+ }
+
+ // The microsecond portion is truncated when written as TIMESTAMP_MILLIS.
+ withTable("ts") {
+ withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "true") {
+ sql("create table ts (c1 int, c2 timestamp) using parquet")
+ sql("insert into ts values (1, '2016-01-01 10:11:12.123456')")
+ sql("insert into ts values (2, null)")
+ sql("insert into ts values (3, '1965-01-01 10:11:12.125456')")
+ sql("insert into ts values (4, '1965-01-01 10:11:12.125')")
+ sql("insert into ts values (5, '1965-01-01 10:11:12.1')")
+ sql("insert into ts values (6, '1965-01-01 10:11:12.123456789')")
+ sql("insert into ts values (7, '0001-01-01 00:00:00.000000')")
+ checkAnswer(
+ sql("select * from ts"),
+ Seq(
+ Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")),
+ Row(2, null),
+ Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")),
+ Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")),
+ Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")),
+ Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")),
+ Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000"))))
+
+ // Read timestamps that were encoded as TIMESTAMP_MILLIS annotated as INT64
+ // with PARQUET_INT64_AS_TIMESTAMP_MILLIS set to false.
+ withSQLConf(SQLConf.PARQUET_INT64_AS_TIMESTAMP_MILLIS.key -> "false") {
+ checkAnswer(
+ sql("select * from ts"),
+ Seq(
+ Row(1, Timestamp.valueOf("2016-01-01 10:11:12.123")),
+ Row(2, null),
+ Row(3, Timestamp.valueOf("1965-01-01 10:11:12.125")),
+ Row(4, Timestamp.valueOf("1965-01-01 10:11:12.125")),
+ Row(5, Timestamp.valueOf("1965-01-01 10:11:12.1")),
+ Row(6, Timestamp.valueOf("1965-01-01 10:11:12.123")),
+ Row(7, Timestamp.valueOf("0001-01-01 00:00:00.000"))))
+ }
+ }
+ }
+ }
+
test("Enabling/disabling merging partfiles when merging parquet schema") {
def testSchemaMerging(expectedColumnNumber: Int): Unit = {
withTempDir { dir =>
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
index 6aa940afbb2c4..ce992674d719f 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetSchemaSuite.scala
@@ -53,11 +53,13 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext {
parquetSchema: String,
binaryAsString: Boolean,
int96AsTimestamp: Boolean,
- writeLegacyParquetFormat: Boolean): Unit = {
+ writeLegacyParquetFormat: Boolean,
+ int64AsTimestampMillis: Boolean = false): Unit = {
val converter = new ParquetSchemaConverter(
assumeBinaryIsString = binaryAsString,
assumeInt96IsTimestamp = int96AsTimestamp,
- writeLegacyParquetFormat = writeLegacyParquetFormat)
+ writeLegacyParquetFormat = writeLegacyParquetFormat,
+ writeTimestampInMillis = int64AsTimestampMillis)
test(s"sql <= parquet: $testName") {
val actual = converter.convert(MessageTypeParser.parseMessageType(parquetSchema))
@@ -77,11 +79,13 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext {
parquetSchema: String,
binaryAsString: Boolean,
int96AsTimestamp: Boolean,
- writeLegacyParquetFormat: Boolean): Unit = {
+ writeLegacyParquetFormat: Boolean,
+ int64AsTimestampMillis: Boolean = false): Unit = {
val converter = new ParquetSchemaConverter(
assumeBinaryIsString = binaryAsString,
assumeInt96IsTimestamp = int96AsTimestamp,
- writeLegacyParquetFormat = writeLegacyParquetFormat)
+ writeLegacyParquetFormat = writeLegacyParquetFormat,
+ writeTimestampInMillis = int64AsTimestampMillis)
test(s"sql => parquet: $testName") {
val actual = converter.convert(sqlSchema)
@@ -97,7 +101,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext {
parquetSchema: String,
binaryAsString: Boolean,
int96AsTimestamp: Boolean,
- writeLegacyParquetFormat: Boolean): Unit = {
+ writeLegacyParquetFormat: Boolean,
+ int64AsTimestampMillis: Boolean = false): Unit = {
testCatalystToParquet(
testName,
@@ -105,7 +110,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext {
parquetSchema,
binaryAsString,
int96AsTimestamp,
- writeLegacyParquetFormat)
+ writeLegacyParquetFormat,
+ int64AsTimestampMillis)
testParquetToCatalyst(
testName,
@@ -113,7 +119,8 @@ abstract class ParquetSchemaTest extends ParquetTest with SharedSQLContext {
parquetSchema,
binaryAsString,
int96AsTimestamp,
- writeLegacyParquetFormat)
+ writeLegacyParquetFormat,
+ int64AsTimestampMillis)
}
}
@@ -965,6 +972,18 @@ class ParquetSchemaSuite extends ParquetSchemaTest {
int96AsTimestamp = true,
writeLegacyParquetFormat = true)
+ testSchema(
+ "Timestamp written and read as INT64 with TIMESTAMP_MILLIS",
+ StructType(Seq(StructField("f1", TimestampType))),
+ """message root {
+ | optional INT64 f1 (TIMESTAMP_MILLIS);
+ |}
+ """.stripMargin,
+ binaryAsString = true,
+ int96AsTimestamp = false,
+ writeLegacyParquetFormat = true,
+ int64AsTimestampMillis = true)
+
private def testSchemaClipping(
testName: String,
parquetSchema: String,
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
index 9742b3b2d5c29..6469e501c1f68 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/internal/CatalogSuite.scala
@@ -102,6 +102,11 @@ class CatalogSuite
assert(col.isPartition == tableMetadata.partitionColumnNames.contains(col.name))
assert(col.isBucket == bucketColumnNames.contains(col.name))
}
+
+ dbName.foreach { db =>
+ val expected = columns.collect().map(_.name).toSet
+ assert(spark.catalog.listColumns(s"$db.$tableName").collect().map(_.name).toSet == expected)
+ }
}
override def afterEach(): Unit = {
@@ -345,6 +350,7 @@ class CatalogSuite
// Find a qualified table
assert(spark.catalog.getTable(db, "tbl_y").name === "tbl_y")
+ assert(spark.catalog.getTable(s"$db.tbl_y").name === "tbl_y")
// Find an unqualified table using the current database
intercept[AnalysisException](spark.catalog.getTable("tbl_y"))
@@ -378,6 +384,11 @@ class CatalogSuite
assert(fn2.database === db)
assert(!fn2.isTemporary)
+ val fn2WithQualifiedName = spark.catalog.getFunction(s"$db.fn2")
+ assert(fn2WithQualifiedName.name === "fn2")
+ assert(fn2WithQualifiedName.database === db)
+ assert(!fn2WithQualifiedName.isTemporary)
+
// Find an unqualified function using the current database
intercept[AnalysisException](spark.catalog.getFunction("fn2"))
spark.catalog.setCurrentDatabase(db)
@@ -403,6 +414,7 @@ class CatalogSuite
assert(!spark.catalog.tableExists("tbl_x"))
assert(!spark.catalog.tableExists("tbl_y"))
assert(!spark.catalog.tableExists(db, "tbl_y"))
+ assert(!spark.catalog.tableExists(s"$db.tbl_y"))
// Create objects.
createTempTable("tbl_x")
@@ -413,11 +425,15 @@ class CatalogSuite
// Find a qualified table
assert(spark.catalog.tableExists(db, "tbl_y"))
+ assert(spark.catalog.tableExists(s"$db.tbl_y"))
// Find an unqualified table using the current database
assert(!spark.catalog.tableExists("tbl_y"))
spark.catalog.setCurrentDatabase(db)
assert(spark.catalog.tableExists("tbl_y"))
+
+ // Unable to find the table, although the temp view with the given name exists
+ assert(!spark.catalog.tableExists(db, "tbl_x"))
}
}
}
@@ -429,6 +445,7 @@ class CatalogSuite
assert(!spark.catalog.functionExists("fn1"))
assert(!spark.catalog.functionExists("fn2"))
assert(!spark.catalog.functionExists(db, "fn2"))
+ assert(!spark.catalog.functionExists(s"$db.fn2"))
// Create objects.
createTempFunction("fn1")
@@ -439,11 +456,15 @@ class CatalogSuite
// Find a qualified function
assert(spark.catalog.functionExists(db, "fn2"))
+ assert(spark.catalog.functionExists(s"$db.fn2"))
// Find an unqualified function using the current database
assert(!spark.catalog.functionExists("fn2"))
spark.catalog.setCurrentDatabase(db)
assert(spark.catalog.functionExists("fn2"))
+
+ // Unable to find the function, although the temp function with the given name exists
+ assert(!spark.catalog.functionExists(db, "fn1"))
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
index 4a02277631f14..5bd36ec25ccb0 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala
@@ -806,10 +806,6 @@ class JDBCSuite extends SparkFunSuite
sql(s"DESC FORMATTED $tableName").collect().foreach { r =>
assert(!r.toString().contains(password))
}
-
- sql(s"DESC EXTENDED $tableName").collect().foreach { r =>
- assert(!r.toString().contains(password))
- }
}
}
}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
deleted file mode 100644
index 674463feca4db..0000000000000
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DDLTestSuite.scala
+++ /dev/null
@@ -1,123 +0,0 @@
-/*
-* Licensed to the Apache Software Foundation (ASF) under one or more
-* contributor license agreements. See the NOTICE file distributed with
-* this work for additional information regarding copyright ownership.
-* The ASF licenses this file to You under the Apache License, Version 2.0
-* (the "License"); you may not use this file except in compliance with
-* the License. You may obtain a copy of the License at
-*
-* http://www.apache.org/licenses/LICENSE-2.0
-*
-* Unless required by applicable law or agreed to in writing, software
-* distributed under the License is distributed on an "AS IS" BASIS,
-* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
-* See the License for the specific language governing permissions and
-* limitations under the License.
-*/
-
-package org.apache.spark.sql.sources
-
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.test.SharedSQLContext
-import org.apache.spark.sql.types._
-import org.apache.spark.unsafe.types.UTF8String
-
-class DDLScanSource extends RelationProvider {
- override def createRelation(
- sqlContext: SQLContext,
- parameters: Map[String, String]): BaseRelation = {
- SimpleDDLScan(
- parameters("from").toInt,
- parameters("TO").toInt,
- parameters("Table"))(sqlContext.sparkSession)
- }
-}
-
-case class SimpleDDLScan(
- from: Int,
- to: Int,
- table: String)(@transient val sparkSession: SparkSession)
- extends BaseRelation with TableScan {
-
- override def sqlContext: SQLContext = sparkSession.sqlContext
-
- override def schema: StructType =
- StructType(Seq(
- StructField("intType", IntegerType, nullable = false).withComment(s"test comment $table"),
- StructField("stringType", StringType, nullable = false),
- StructField("dateType", DateType, nullable = false),
- StructField("timestampType", TimestampType, nullable = false),
- StructField("doubleType", DoubleType, nullable = false),
- StructField("bigintType", LongType, nullable = false),
- StructField("tinyintType", ByteType, nullable = false),
- StructField("decimalType", DecimalType.USER_DEFAULT, nullable = false),
- StructField("fixedDecimalType", DecimalType(5, 1), nullable = false),
- StructField("binaryType", BinaryType, nullable = false),
- StructField("booleanType", BooleanType, nullable = false),
- StructField("smallIntType", ShortType, nullable = false),
- StructField("floatType", FloatType, nullable = false),
- StructField("mapType", MapType(StringType, StringType)),
- StructField("arrayType", ArrayType(StringType)),
- StructField("structType",
- StructType(StructField("f1", StringType) :: StructField("f2", IntegerType) :: Nil
- )
- )
- ))
-
- override def needConversion: Boolean = false
-
- override def buildScan(): RDD[Row] = {
- // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row]
- sparkSession.sparkContext.parallelize(from to to).map { e =>
- InternalRow(UTF8String.fromString(s"people$e"), e * 2)
- }.asInstanceOf[RDD[Row]]
- }
-}
-
-class DDLTestSuite extends DataSourceTest with SharedSQLContext {
- protected override lazy val sql = spark.sql _
-
- override def beforeAll(): Unit = {
- super.beforeAll()
- sql(
- """
- |CREATE OR REPLACE TEMPORARY VIEW ddlPeople
- |USING org.apache.spark.sql.sources.DDLScanSource
- |OPTIONS (
- | From '1',
- | To '10',
- | Table 'test1'
- |)
- """.stripMargin)
- }
-
- sqlTest(
- "describe ddlPeople",
- Seq(
- Row("intType", "int", "test comment test1"),
- Row("stringType", "string", null),
- Row("dateType", "date", null),
- Row("timestampType", "timestamp", null),
- Row("doubleType", "double", null),
- Row("bigintType", "bigint", null),
- Row("tinyintType", "tinyint", null),
- Row("decimalType", "decimal(10,0)", null),
- Row("fixedDecimalType", "decimal(5,1)", null),
- Row("binaryType", "binary", null),
- Row("booleanType", "boolean", null),
- Row("smallIntType", "smallint", null),
- Row("floatType", "float", null),
- Row("mapType", "map", null),
- Row("arrayType", "array", null),
- Row("structType", "struct", null)
- ))
-
- test("SPARK-7686 DescribeCommand should have correct physical plan output attributes") {
- val attributes = sql("describe ddlPeople")
- .queryExecution.executedPlan.output
- assert(attributes.map(_.name) === Seq("col_name", "data_type", "comment"))
- assert(attributes.map(_.dataType).toSet === Set(StringType))
- }
-}
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
index 448adcf11d656..b16c9f8fc96b2 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceAnalysisSuite.scala
@@ -21,10 +21,10 @@ import org.scalatest.BeforeAndAfterAll
import org.apache.spark.SparkFunSuite
import org.apache.spark.sql.AnalysisException
-import org.apache.spark.sql.catalyst.SimpleCatalystConf
import org.apache.spark.sql.catalyst.dsl.expressions._
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Cast, Expression, Literal}
import org.apache.spark.sql.execution.datasources.DataSourceAnalysis
+import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{IntegerType, StructType}
class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
@@ -49,7 +49,7 @@ class DataSourceAnalysisSuite extends SparkFunSuite with BeforeAndAfterAll {
}
Seq(true, false).foreach { caseSensitive =>
- val rule = DataSourceAnalysis(SimpleCatalystConf(caseSensitive))
+ val rule = DataSourceAnalysis(new SQLConf().copy(SQLConf.CASE_SENSITIVE -> caseSensitive))
test(
s"convertStaticPartitions only handle INSERT having at least static partitions " +
s"(caseSensitive: $caseSensitive)") {
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
index cc77d3c4b91ac..80868fff897fd 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/sources/DataSourceTest.scala
@@ -17,7 +17,11 @@
package org.apache.spark.sql.sources
+import org.apache.spark.rdd.RDD
import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.types._
+import org.apache.spark.unsafe.types.UTF8String
private[sql] abstract class DataSourceTest extends QueryTest {
@@ -28,3 +32,55 @@ private[sql] abstract class DataSourceTest extends QueryTest {
}
}
+
+class DDLScanSource extends RelationProvider {
+ override def createRelation(
+ sqlContext: SQLContext,
+ parameters: Map[String, String]): BaseRelation = {
+ SimpleDDLScan(
+ parameters("from").toInt,
+ parameters("TO").toInt,
+ parameters("Table"))(sqlContext.sparkSession)
+ }
+}
+
+case class SimpleDDLScan(
+ from: Int,
+ to: Int,
+ table: String)(@transient val sparkSession: SparkSession)
+ extends BaseRelation with TableScan {
+
+ override def sqlContext: SQLContext = sparkSession.sqlContext
+
+ override def schema: StructType =
+ StructType(Seq(
+ StructField("intType", IntegerType, nullable = false).withComment(s"test comment $table"),
+ StructField("stringType", StringType, nullable = false),
+ StructField("dateType", DateType, nullable = false),
+ StructField("timestampType", TimestampType, nullable = false),
+ StructField("doubleType", DoubleType, nullable = false),
+ StructField("bigintType", LongType, nullable = false),
+ StructField("tinyintType", ByteType, nullable = false),
+ StructField("decimalType", DecimalType.USER_DEFAULT, nullable = false),
+ StructField("fixedDecimalType", DecimalType(5, 1), nullable = false),
+ StructField("binaryType", BinaryType, nullable = false),
+ StructField("booleanType", BooleanType, nullable = false),
+ StructField("smallIntType", ShortType, nullable = false),
+ StructField("floatType", FloatType, nullable = false),
+ StructField("mapType", MapType(StringType, StringType)),
+ StructField("arrayType", ArrayType(StringType)),
+ StructField("structType",
+ StructType(StructField("f1", StringType) :: StructField("f2", IntegerType) :: Nil
+ )
+ )
+ ))
+
+ override def needConversion: Boolean = false
+
+ override def buildScan(): RDD[Row] = {
+ // Rely on a type erasure hack to pass RDD[InternalRow] back as RDD[Row]
+ sparkSession.sparkContext.parallelize(from to to).map { e =>
+ InternalRow(UTF8String.fromString(s"people$e"), e * 2)
+ }.asInstanceOf[RDD[Row]]
+ }
+}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
index 55e02acfa4ce3..b554694815571 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
@@ -767,9 +767,6 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
sessionState.refreshTable(tableName)
val actualSchema = table(tableName).schema
assert(schema === actualSchema)
-
- // Checks the DESCRIBE output.
- checkAnswer(sql("DESCRIBE spark6655"), Row("int", "int", null) :: Nil)
}
}
@@ -1381,7 +1378,10 @@ class MetastoreDataSourcesSuite extends QueryTest with SQLTestUtils with TestHiv
checkAnswer(spark.table("old"), Row(1, "a"))
- checkAnswer(sql("DESC old"), Row("i", "int", null) :: Row("j", "string", null) :: Nil)
+ val expectedSchema = StructType(Seq(
+ StructField("i", IntegerType, nullable = true),
+ StructField("j", StringType, nullable = true)))
+ assert(table("old").schema === expectedSchema)
}
}
}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
index 536ca8fd9d45d..e45cf977bfaa2 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
@@ -207,6 +207,7 @@ abstract class HiveComparisonTest
// This list contains indicators for those lines which do not have actual results and we
// want to ignore.
lazy val ignoredLineIndicators = Seq(
+ "# Detailed Table Information",
"# Partition Information",
"# col_name"
)
@@ -358,7 +359,7 @@ abstract class HiveComparisonTest
stringToFile(new File(failedDirectory, testCaseName), errorMessage + consoleTestCase)
fail(errorMessage)
}
- }.toSeq
+ }
(queryList, hiveResults, catalystResults).zipped.foreach {
case (query, hive, (hiveQuery, catalyst)) =>
@@ -369,6 +370,7 @@ abstract class HiveComparisonTest
if ((!hiveQuery.logical.isInstanceOf[ExplainCommand]) &&
(!hiveQuery.logical.isInstanceOf[ShowFunctionsCommand]) &&
(!hiveQuery.logical.isInstanceOf[DescribeFunctionCommand]) &&
+ (!hiveQuery.logical.isInstanceOf[DescribeTableCommand]) &&
preparedHive != catalyst) {
val hivePrintOut = s"== HIVE - ${preparedHive.size} row(s) ==" +: preparedHive
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
index f0a995c274b64..3906968aaff10 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala
@@ -708,23 +708,6 @@ class HiveDDLSuite
}
}
- test("desc table for Hive table") {
- withTable("tab1") {
- val tabName = "tab1"
- sql(s"CREATE TABLE $tabName(c1 int)")
-
- assert(sql(s"DESC $tabName").collect().length == 1)
-
- assert(
- sql(s"DESC FORMATTED $tabName").collect()
- .exists(_.getString(0) == "# Storage Information"))
-
- assert(
- sql(s"DESC EXTENDED $tabName").collect()
- .exists(_.getString(0) == "# Detailed Table Information"))
- }
- }
-
test("desc table for Hive table - partitioned table") {
withTable("tbl") {
sql("CREATE TABLE tbl(a int) PARTITIONED BY (b int)")
@@ -741,23 +724,6 @@ class HiveDDLSuite
}
}
- test("desc formatted table for permanent view") {
- withTable("tbl") {
- withView("view1") {
- sql("CREATE TABLE tbl(a int)")
- sql("CREATE VIEW view1 AS SELECT * FROM tbl")
- assert(sql("DESC FORMATTED view1").collect().containsSlice(
- Seq(
- Row("# View Information", "", ""),
- Row("View Text:", "SELECT * FROM tbl", ""),
- Row("View Default Database:", "default", ""),
- Row("View Query Output Columns:", "[a]", "")
- )
- ))
- }
- }
- }
-
test("desc table for data source table using Hive Metastore") {
assume(spark.sparkContext.conf.get(CATALOG_IMPLEMENTATION) == "hive")
val tabName = "tab1"
@@ -766,7 +732,7 @@ class HiveDDLSuite
checkAnswer(
sql(s"DESC $tabName").select("col_name", "data_type", "comment"),
- Row("a", "int", "test")
+ Row("# col_name", "data_type", "comment") :: Row("a", "int", "test") :: Nil
)
}
}
@@ -1218,23 +1184,6 @@ class HiveDDLSuite
sql(s"SELECT * FROM ${targetTable.identifier}"))
}
- test("desc table for data source table") {
- withTable("tab1") {
- val tabName = "tab1"
- spark.range(1).write.format("json").saveAsTable(tabName)
-
- assert(sql(s"DESC $tabName").collect().length == 1)
-
- assert(
- sql(s"DESC FORMATTED $tabName").collect()
- .exists(_.getString(0) == "# Storage Information"))
-
- assert(
- sql(s"DESC EXTENDED $tabName").collect()
- .exists(_.getString(0) == "# Detailed Table Information"))
- }
- }
-
test("create table with the same name as an index table") {
val tabName = "tab1"
val indexName = tabName + "_index"
@@ -1320,46 +1269,6 @@ class HiveDDLSuite
}
}
- test("desc table for data source table - partitioned bucketed table") {
- withTable("t1") {
- spark
- .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
- .bucketBy(2, "b").sortBy("c").partitionBy("d")
- .saveAsTable("t1")
-
- val formattedDesc = sql("DESC FORMATTED t1").collect()
-
- assert(formattedDesc.containsSlice(
- Seq(
- Row("a", "bigint", null),
- Row("b", "bigint", null),
- Row("c", "bigint", null),
- Row("d", "bigint", null),
- Row("# Partition Information", "", ""),
- Row("# col_name", "data_type", "comment"),
- Row("d", "bigint", null),
- Row("", "", ""),
- Row("# Detailed Table Information", "", ""),
- Row("Database:", "default", "")
- )
- ))
-
- assert(formattedDesc.containsSlice(
- Seq(
- Row("Table Type:", "MANAGED", "")
- )
- ))
-
- assert(formattedDesc.containsSlice(
- Seq(
- Row("Num Buckets:", "2", ""),
- Row("Bucket Columns:", "[b]", ""),
- Row("Sort Columns:", "[c]", "")
- )
- ))
- }
- }
-
test("datasource and statistics table property keys are not allowed") {
import org.apache.spark.sql.hive.HiveExternalCatalog.DATASOURCE_PREFIX
import org.apache.spark.sql.hive.HiveExternalCatalog.STATISTICS_PREFIX
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala
deleted file mode 100644
index 0e89e990e564e..0000000000000
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveOperatorQueryableSuite.scala
+++ /dev/null
@@ -1,53 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements. See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.apache.spark.sql.hive.execution
-
-import org.apache.spark.sql.{QueryTest, Row}
-import org.apache.spark.sql.hive.test.TestHiveSingleton
-
-/**
- * A set of tests that validates commands can also be queried by like a table
- */
-class HiveOperatorQueryableSuite extends QueryTest with TestHiveSingleton {
- import spark._
-
- test("SPARK-5324 query result of describe command") {
- hiveContext.loadTestTable("src")
-
- // Creates a temporary view with the output of a describe command
- sql("desc src").createOrReplaceTempView("mydesc")
- checkAnswer(
- sql("desc mydesc"),
- Seq(
- Row("col_name", "string", "name of the column"),
- Row("data_type", "string", "data type of the column"),
- Row("comment", "string", "comment of the column")))
-
- checkAnswer(
- sql("select * from mydesc"),
- Seq(
- Row("key", "int", null),
- Row("value", "string", null)))
-
- checkAnswer(
- sql("select col_name, data_type, comment from mydesc"),
- Seq(
- Row("key", "int", null),
- Row("value", "string", null)))
- }
-}
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
index dd278f683a3cd..65a902fc5438e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
@@ -789,62 +789,6 @@ class HiveQuerySuite extends HiveComparisonTest with SQLTestUtils with BeforeAnd
assert(Try(q0.count()).isSuccess)
}
- test("DESCRIBE commands") {
- sql(s"CREATE TABLE test_describe_commands1 (key INT, value STRING) PARTITIONED BY (dt STRING)")
-
- sql(
- """FROM src INSERT OVERWRITE TABLE test_describe_commands1 PARTITION (dt='2008-06-08')
- |SELECT key, value
- """.stripMargin)
-
- // Describe a table
- assertResult(
- Array(
- Row("key", "int", null),
- Row("value", "string", null),
- Row("dt", "string", null),
- Row("# Partition Information", "", ""),
- Row("# col_name", "data_type", "comment"),
- Row("dt", "string", null))
- ) {
- sql("DESCRIBE test_describe_commands1")
- .select('col_name, 'data_type, 'comment)
- .collect()
- }
-
- // Describe a table with a fully qualified table name
- assertResult(
- Array(
- Row("key", "int", null),
- Row("value", "string", null),
- Row("dt", "string", null),
- Row("# Partition Information", "", ""),
- Row("# col_name", "data_type", "comment"),
- Row("dt", "string", null))
- ) {
- sql("DESCRIBE default.test_describe_commands1")
- .select('col_name, 'data_type, 'comment)
- .collect()
- }
-
- // Describe a temporary view.
- val testData =
- TestHive.sparkContext.parallelize(
- TestData(1, "str1") ::
- TestData(1, "str2") :: Nil)
- testData.toDF().createOrReplaceTempView("test_describe_commands2")
-
- assertResult(
- Array(
- Row("a", "int", null),
- Row("b", "string", null))
- ) {
- sql("DESCRIBE test_describe_commands2")
- .select('col_name, 'data_type, 'comment)
- .collect()
- }
- }
-
test("SPARK-2263: Insert Map values") {
sql("CREATE TABLE m(value MAP)")
sql("INSERT OVERWRITE TABLE m SELECT MAP(key, value) FROM src LIMIT 10")
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
index 55ff4bb115e59..d012797e19926 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
@@ -363,79 +363,6 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}
}
- test("describe partition") {
- withTable("partitioned_table") {
- sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
- sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
-
- checkKeywordsExist(sql("DESC partitioned_table PARTITION (c='Us', d=1)"),
- "# Partition Information",
- "# col_name")
-
- checkKeywordsExist(sql("DESC EXTENDED partitioned_table PARTITION (c='Us', d=1)"),
- "# Partition Information",
- "# col_name",
- "Detailed Partition Information CatalogPartition(",
- "Partition Values: [c=Us, d=1]",
- "Storage(Location:",
- "Partition Parameters")
-
- checkKeywordsExist(sql("DESC FORMATTED partitioned_table PARTITION (c='Us', d=1)"),
- "# Partition Information",
- "# col_name",
- "# Detailed Partition Information",
- "Partition Value:",
- "Database:",
- "Table:",
- "Location:",
- "Partition Parameters:",
- "# Storage Information")
- }
- }
-
- test("describe partition - error handling") {
- withTable("partitioned_table", "datasource_table") {
- sql("CREATE TABLE partitioned_table (a STRING, b INT) PARTITIONED BY (c STRING, d STRING)")
- sql("ALTER TABLE partitioned_table ADD PARTITION (c='Us', d=1)")
-
- val m = intercept[NoSuchPartitionException] {
- sql("DESC partitioned_table PARTITION (c='Us', d=2)")
- }.getMessage()
- assert(m.contains("Partition not found in table"))
-
- val m2 = intercept[AnalysisException] {
- sql("DESC partitioned_table PARTITION (c='Us')")
- }.getMessage()
- assert(m2.contains("Partition spec is invalid"))
-
- val m3 = intercept[ParseException] {
- sql("DESC partitioned_table PARTITION (c='Us', d)")
- }.getMessage()
- assert(m3.contains("PARTITION specification is incomplete: `d`"))
-
- spark
- .range(1).select('id as 'a, 'id as 'b, 'id as 'c, 'id as 'd).write
- .partitionBy("d")
- .saveAsTable("datasource_table")
-
- sql("DESC datasource_table PARTITION (d=0)")
-
- val m5 = intercept[AnalysisException] {
- spark.range(10).select('id as 'a, 'id as 'b).createTempView("view1")
- sql("DESC view1 PARTITION (c='Us', d=1)")
- }.getMessage()
- assert(m5.contains("DESC PARTITION is not allowed on a temporary view"))
-
- withView("permanent_view") {
- val m = intercept[AnalysisException] {
- sql("CREATE VIEW permanent_view AS SELECT * FROM partitioned_table")
- sql("DESC permanent_view PARTITION (c='Us', d=1)")
- }.getMessage()
- assert(m.contains("DESC PARTITION is not allowed on a view"))
- }
- }
- }
-
test("SPARK-5371: union with null and sum") {
val df = Seq((1, 1)).toDF("c1", "c2")
df.createOrReplaceTempView("table1")
@@ -676,7 +603,7 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
}
test("CTAS with serde") {
- sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value").collect()
+ sql("CREATE TABLE ctas1 AS SELECT key k, value FROM src ORDER BY k, value")
sql(
"""CREATE TABLE ctas2
| ROW FORMAT SERDE "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"
@@ -686,86 +613,76 @@ class SQLQuerySuite extends QueryTest with SQLTestUtils with TestHiveSingleton {
| AS
| SELECT key, value
| FROM src
- | ORDER BY key, value""".stripMargin).collect()
+ | ORDER BY key, value""".stripMargin)
+
+ val storageCtas2 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("ctas2")).storage
+ assert(storageCtas2.inputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileInputFormat"))
+ assert(storageCtas2.outputFormat == Some("org.apache.hadoop.hive.ql.io.RCFileOutputFormat"))
+ assert(storageCtas2.serde == Some("org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe"))
+
sql(
"""CREATE TABLE ctas3
| ROW FORMAT DELIMITED FIELDS TERMINATED BY ',' LINES TERMINATED BY '\012'
| STORED AS textfile AS
| SELECT key, value
| FROM src
- | ORDER BY key, value""".stripMargin).collect()
+ | ORDER BY key, value""".stripMargin)
// the table schema may like (key: integer, value: string)
sql(
"""CREATE TABLE IF NOT EXISTS ctas4 AS
- | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin).collect()
+ | SELECT 1 AS key, value FROM src LIMIT 1""".stripMargin)
// do nothing cause the table ctas4 already existed.
sql(
"""CREATE TABLE IF NOT EXISTS ctas4 AS
- | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect()
+ | SELECT key, value FROM src ORDER BY key, value""".stripMargin)
checkAnswer(
sql("SELECT k, value FROM ctas1 ORDER BY k, value"),
- sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
+ sql("SELECT key, value FROM src ORDER BY key, value"))
checkAnswer(
sql("SELECT key, value FROM ctas2 ORDER BY key, value"),
sql(
"""
SELECT key, value
FROM src
- ORDER BY key, value""").collect().toSeq)
+ ORDER BY key, value"""))
checkAnswer(
sql("SELECT key, value FROM ctas3 ORDER BY key, value"),
sql(
"""
SELECT key, value
FROM src
- ORDER BY key, value""").collect().toSeq)
+ ORDER BY key, value"""))
intercept[AnalysisException] {
sql(
"""CREATE TABLE ctas4 AS
- | SELECT key, value FROM src ORDER BY key, value""".stripMargin).collect()
+ | SELECT key, value FROM src ORDER BY key, value""".stripMargin)
}
checkAnswer(
sql("SELECT key, value FROM ctas4 ORDER BY key, value"),
sql("SELECT key, value FROM ctas4 LIMIT 1").collect().toSeq)
- /*
- Disabled because our describe table does not output the serde information right now.
- checkKeywordsExist(sql("DESC EXTENDED ctas2"),
- "name:key", "type:string", "name:value", "ctas2",
- "org.apache.hadoop.hive.ql.io.RCFileInputFormat",
- "org.apache.hadoop.hive.ql.io.RCFileOutputFormat",
- "org.apache.hadoop.hive.serde2.columnar.ColumnarSerDe",
- "serde_p1=p1", "serde_p2=p2", "tbl_p1=p11", "tbl_p2=p22", "MANAGED_TABLE"
- )
- */
-
sql(
"""CREATE TABLE ctas5
| STORED AS parquet AS
| SELECT key, value
| FROM src
- | ORDER BY key, value""".stripMargin).collect()
+ | ORDER BY key, value""".stripMargin)
+ val storageCtas5 = spark.sessionState.catalog.getTableMetadata(TableIdentifier("ctas5")).storage
+ assert(storageCtas5.inputFormat ==
+ Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat"))
+ assert(storageCtas5.outputFormat ==
+ Some("org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat"))
+ assert(storageCtas5.serde ==
+ Some("org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe"))
- /*
- Disabled because our describe table does not output the serde information right now.
- withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") {
- checkKeywordsExist(sql("DESC EXTENDED ctas5"),
- "name:key", "type:string", "name:value", "ctas5",
- "org.apache.hadoop.hive.ql.io.parquet.MapredParquetInputFormat",
- "org.apache.hadoop.hive.ql.io.parquet.MapredParquetOutputFormat",
- "org.apache.hadoop.hive.ql.io.parquet.serde.ParquetHiveSerDe",
- "MANAGED_TABLE"
- )
- }
- */
// use the Hive SerDe for parquet tables
withSQLConf(HiveUtils.CONVERT_METASTORE_PARQUET.key -> "false") {
checkAnswer(
sql("SELECT key, value FROM ctas5 ORDER BY key, value"),
- sql("SELECT key, value FROM src ORDER BY key, value").collect().toSeq)
+ sql("SELECT key, value FROM src ORDER BY key, value"))
}
}