You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@spark.apache.org by pw...@apache.org on 2014/03/26 23:36:27 UTC
[1/2] Unified package definition format in Spark SQL
Repository: spark
Updated Branches:
refs/heads/master a0853a39e -> 345825d97
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
index 3bcf586..fc5057b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveContext.scala
@@ -32,7 +32,7 @@ import org.apache.spark.SparkContext
import org.apache.spark.rdd.RDD
import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog}
import org.apache.spark.sql.catalyst.expressions.GenericRow
-import org.apache.spark.sql.catalyst.plans.logical.{BaseRelation, LogicalPlan, LowerCaseSchema}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, LowerCaseSchema}
import org.apache.spark.sql.catalyst.plans.logical.{NativeCommand, ExplainCommand}
import org.apache.spark.sql.catalyst.types._
import org.apache.spark.sql.execution._
@@ -86,7 +86,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
val input = new java.io.InputStream {
val iterator = (start ++ end).iterator
- def read(): Int = if (iterator.hasNext) iterator.next else -1
+ def read(): Int = if (iterator.hasNext) iterator.next() else -1
}
val reader = new BufferedReader(new InputStreamReader(input))
val stringBuilder = new StringBuilder
@@ -148,24 +148,24 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
SessionState.start(sessionState)
- if (proc.isInstanceOf[Driver]) {
- val driver: Driver = proc.asInstanceOf[Driver]
- driver.init()
+ proc match {
+ case driver: Driver =>
+ driver.init()
- val results = new JArrayList[String]
- val response: CommandProcessorResponse = driver.run(cmd)
- // Throw an exception if there is an error in query processing.
- if (response.getResponseCode != 0) {
+ val results = new JArrayList[String]
+ val response: CommandProcessorResponse = driver.run(cmd)
+ // Throw an exception if there is an error in query processing.
+ if (response.getResponseCode != 0) {
+ driver.destroy()
+ throw new QueryExecutionException(response.getErrorMessage)
+ }
+ driver.setMaxRows(maxRows)
+ driver.getResults(results)
driver.destroy()
- throw new QueryExecutionException(response.getErrorMessage)
- }
- driver.setMaxRows(maxRows)
- driver.getResults(results)
- driver.destroy()
- results
- } else {
- sessionState.out.println(tokens(0) + " " + cmd_1)
- Seq(proc.run(cmd_1).getResponseCode.toString)
+ results
+ case _ =>
+ sessionState.out.println(tokens(0) + " " + cmd_1)
+ Seq(proc.run(cmd_1).getResponseCode.toString)
}
} catch {
case e: Exception =>
@@ -226,7 +226,7 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) {
sparkContext.parallelize(asRows, 1)
}
case _ =>
- executedPlan.execute.map(_.copy())
+ executedPlan.execute().map(_.copy())
}
protected val primitiveTypes =
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
index 1667a21..4f83536 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import scala.util.parsing.combinator.RegexParsers
@@ -27,14 +26,15 @@ import org.apache.hadoop.hive.ql.plan.TableDesc
import org.apache.hadoop.hive.ql.session.SessionState
import org.apache.hadoop.hive.serde2.Deserializer
-
-import org.apache.spark.sql.catalyst.analysis.{Catalog, EliminateAnalysisOperators}
+import org.apache.spark.sql.Logging
+import org.apache.spark.sql.catalyst.analysis.{EliminateAnalysisOperators, Catalog}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
import org.apache.spark.sql.catalyst.types._
+/* Implicit conversions */
import scala.collection.JavaConversions._
class HiveMetastoreCatalog(hive: HiveContext) extends Catalog with Logging {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
index 8e76a73..f4b6138 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveQl.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import org.apache.hadoop.hive.ql.lib.Node
import org.apache.hadoop.hive.ql.parse._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
index 1410672..3ca1d93 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveStrategies.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
+import org.apache.spark.sql.SQLContext
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning._
import org.apache.spark.sql.catalyst.plans._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
index dc4181e..2610100 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/ScriptTransformation.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import java.io.{BufferedReader, InputStreamReader}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
index 99dc85e..ca53113 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TableReader.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import org.apache.hadoop.conf.Configuration
import org.apache.hadoop.fs.{Path, PathFilter}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
index a26b0ff..bc3447b 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/TestHive.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import java.io.File
import java.util.{Set => JavaSet}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
index 78f69e7..e2d9d8d 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveOperators.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import org.apache.hadoop.hive.common.`type`.{HiveDecimal, HiveVarchar}
import org.apache.hadoop.hive.metastore.MetaStoreUtils
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
index 72ccd4f..44901db 100644
--- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveUdfs.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
+package org.apache.spark.sql.hive
import scala.collection.mutable.ArrayBuffer
@@ -29,6 +28,7 @@ import org.apache.hadoop.hive.serde2.objectinspector.primitive._
import org.apache.hadoop.hive.serde2.{io => hiveIo}
import org.apache.hadoop.{io => hadoopIo}
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.types
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
index 4b45e69..8488f23 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/BigDataBenchmarkSuite.scala
@@ -15,19 +15,17 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
import java.io.File
+import org.apache.spark.sql.hive.TestHive._
+
/**
* A set of test cases based on the big-data-benchmark.
* https://amplab.cs.berkeley.edu/benchmark/
*/
class BigDataBenchmarkSuite extends HiveComparisonTest {
- import TestHive._
-
val testDataDirectory = new File("target/big-data-benchmark-testdata")
val testTables = Seq(
@@ -123,4 +121,4 @@ class BigDataBenchmarkSuite extends HiveComparisonTest {
|-- SELECT * FROM url_counts_total
""".stripMargin)
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
index 02ee2a0..ac87f2c 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/ConcurrentHiveSuite.scala
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark
-package sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
-import org.scalatest.{FunSuite, BeforeAndAfterAll}
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.hive.TestHiveContext
+import org.scalatest.{BeforeAndAfterAll, FunSuite}
class ConcurrentHiveSuite extends FunSuite with BeforeAndAfterAll {
ignore("multiple instances not supported") {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveComparisonTest.scala
----------------------------------------------------------------------
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 e8fcc27..c7a350e 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
@@ -15,17 +15,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
import java.io._
-import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
-
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.plans.logical.{ExplainCommand, NativeCommand}
import org.apache.spark.sql.catalyst.util._
import org.apache.spark.sql.execution.Sort
+import org.scalatest.{BeforeAndAfterAll, FunSuite, GivenWhenThen}
+import org.apache.spark.sql.hive.TestHive
/**
* Allows the creations of tests that execute the same query against both hive
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
index d5b54fa..f74b0fb 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveCompatibilitySuite.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.hive.TestHive
/**
* Runs the test cases that are included in the hive distribution.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
index 2d2f133..50ab71a 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQueryFileTest.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
import java.io.File
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveQuerySuite.scala
----------------------------------------------------------------------
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 b804634..4b92d16 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
@@ -15,16 +15,14 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
+
+import org.apache.spark.sql.hive.TestHive._
/**
* A set of test cases expressed in Hive QL that are not covered by the tests included in the hive distribution.
*/
class HiveQuerySuite extends HiveComparisonTest {
- import TestHive._
-
createQueryTest("Simple Average",
"SELECT AVG(key) FROM src")
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
index 4bdea21..d77900d 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveResolutionSuite.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
-import TestHive._
+import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.TestHive._
case class Data(a: Int, B: Int, n: Nested)
case class Nested(a: Int, B: Int)
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
index c226492..df9bae9 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveSerDeSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
/**
* A set of tests that validates support for Hive SerDe.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
index bb33583..e030c8e 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTypeCoercionSuite.scala
@@ -21,7 +21,6 @@ package org.apache.spark.sql.hive.execution
* A set of tests that validate type promotion rules.
*/
class HiveTypeCoercionSuite extends HiveComparisonTest {
-
val baseTypes = Seq("1", "1.0", "1L", "1S", "1Y", "'1'")
baseTypes.foreach { i =>
@@ -29,4 +28,4 @@ class HiveTypeCoercionSuite extends HiveComparisonTest {
createQueryTest(s"$i + $j", s"SELECT $i + $j FROM src LIMIT 1")
}
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
index d2f8e5d..1318ac1 100644
--- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
+++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/PruningSuite.scala
@@ -15,11 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package hive
-package execution
+package org.apache.spark.sql.hive.execution
-import org.apache.spark.sql.hive.TestHive
+import org.apache.spark.sql.hive.{HiveTableScan, TestHive}
/* Implicit conversions */
import scala.collection.JavaConversions._
[2/2] git commit: Unified package definition format in Spark SQL
Posted by pw...@apache.org.
Unified package definition format in Spark SQL
According to discussions in comments of PR #208, this PR unifies package definition format in Spark SQL.
Some broken links in ScalaDoc and typos detected along the way are also fixed.
Author: Cheng Lian <li...@gmail.com>
Closes #225 from liancheng/packageDefinition and squashes the following commits:
75c47b3 [Cheng Lian] Fixed file line length
4f87968 [Cheng Lian] Unified package definition format in Spark SQL
Project: http://git-wip-us.apache.org/repos/asf/spark/repo
Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/345825d9
Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/345825d9
Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/345825d9
Branch: refs/heads/master
Commit: 345825d97987b9eeb2afcf002f815a05ff51fc2e
Parents: a0853a3
Author: Cheng Lian <li...@gmail.com>
Authored: Wed Mar 26 15:36:18 2014 -0700
Committer: Patrick Wendell <pw...@gmail.com>
Committed: Wed Mar 26 15:36:18 2014 -0700
----------------------------------------------------------------------
.../scala/org/apache/spark/Dependency.scala | 5 +--
.../org/apache/spark/api/java/JavaRDDLike.scala | 4 +--
.../spark/sql/catalyst/ScalaReflection.scala | 3 +-
.../spark/sql/catalyst/analysis/Analyzer.scala | 9 +++--
.../spark/sql/catalyst/analysis/Catalog.scala | 4 +--
.../catalyst/analysis/FunctionRegistry.scala | 4 +--
.../catalyst/analysis/HiveTypeCoercion.scala | 8 ++---
.../analysis/MultiInstanceRelation.scala | 3 +-
.../spark/sql/catalyst/analysis/package.scala | 3 +-
.../sql/catalyst/analysis/unresolved.scala | 5 ++-
.../apache/spark/sql/catalyst/dsl/package.scala | 3 +-
.../spark/sql/catalyst/errors/package.scala | 3 +-
.../catalyst/expressions/BoundAttribute.scala | 6 ++--
.../spark/sql/catalyst/expressions/Cast.scala | 4 +--
.../sql/catalyst/expressions/Expression.scala | 9 +++--
.../sql/catalyst/expressions/Projection.scala | 3 +-
.../spark/sql/catalyst/expressions/Rand.scala | 4 +--
.../spark/sql/catalyst/expressions/Row.scala | 4 +--
.../sql/catalyst/expressions/ScalaUdf.scala | 4 +--
.../sql/catalyst/expressions/SortOrder.scala | 4 +--
.../sql/catalyst/expressions/WrapDynamic.scala | 4 +--
.../sql/catalyst/expressions/aggregates.scala | 5 ++-
.../sql/catalyst/expressions/arithmetic.scala | 4 +--
.../sql/catalyst/expressions/complexTypes.scala | 4 +--
.../sql/catalyst/expressions/generators.scala | 5 ++-
.../sql/catalyst/expressions/literals.scala | 4 +--
.../catalyst/expressions/namedExpressions.scala | 5 ++-
.../catalyst/expressions/nullFunctions.scala | 5 ++-
.../sql/catalyst/expressions/package.scala | 5 ++-
.../sql/catalyst/expressions/predicates.scala | 7 ++--
.../catalyst/expressions/stringOperations.scala | 4 +--
.../sql/catalyst/optimizer/Optimizer.scala | 5 +--
.../sql/catalyst/planning/QueryPlanner.scala | 5 ++-
.../spark/sql/catalyst/planning/package.scala | 3 +-
.../spark/sql/catalyst/planning/patterns.scala | 8 ++---
.../spark/sql/catalyst/plans/QueryPlan.scala | 4 +--
.../spark/sql/catalyst/plans/joinTypes.scala | 4 +--
.../catalyst/plans/logical/BaseRelation.scala | 5 +--
.../catalyst/plans/logical/LogicalPlan.scala | 7 ++--
.../plans/logical/ScriptTransformation.scala | 5 +--
.../catalyst/plans/logical/TestRelation.scala | 6 ++--
.../catalyst/plans/logical/basicOperators.scala | 6 ++--
.../catalyst/plans/logical/partitioning.scala | 5 +--
.../spark/sql/catalyst/plans/package.scala | 3 +-
.../catalyst/plans/physical/partitioning.scala | 5 +--
.../apache/spark/sql/catalyst/rules/Rule.scala | 5 ++-
.../spark/sql/catalyst/rules/package.scala | 3 +-
.../spark/sql/catalyst/trees/TreeNode.scala | 4 +--
.../spark/sql/catalyst/trees/package.scala | 5 +--
.../spark/sql/catalyst/types/dataTypes.scala | 4 +--
.../spark/sql/catalyst/types/package.scala | 3 +-
.../spark/sql/catalyst/util/package.scala | 3 +-
.../spark/sql/catalyst/DistributionSuite.scala | 5 ++-
.../sql/catalyst/analysis/AnalysisSuite.scala | 4 +--
.../analysis/HiveTypeCoercionSuite.scala | 4 +--
.../expressions/ExpressionEvaluationSuite.scala | 4 +--
.../optimizer/ConstantFoldingSuite.scala | 6 ++--
.../optimizer/FilterPushdownSuite.scala | 8 ++---
.../sql/catalyst/optimizer/OptimizerTest.scala | 4 +--
.../sql/catalyst/trees/RuleExecutorSuite.scala | 4 +--
.../spark/sql/columnar/ColumnAccessor.scala | 3 +-
.../spark/sql/columnar/ColumnBuilder.scala | 8 ++---
.../apache/spark/sql/columnar/ColumnType.scala | 4 +--
.../sql/columnar/NullableColumnBuilder.scala | 7 ++--
.../columnar/inMemoryColumnarOperators.scala | 10 +++---
.../apache/spark/sql/execution/Exchange.scala | 6 ++--
.../apache/spark/sql/execution/Generate.scala | 5 ++-
.../sql/execution/QueryExecutionException.scala | 3 +-
.../apache/spark/sql/execution/SparkPlan.scala | 13 ++++---
.../sql/execution/SparkSqlSerializer.scala | 3 +-
.../spark/sql/execution/SparkStrategies.scala | 6 ++--
.../apache/spark/sql/execution/aggregates.scala | 3 +-
.../spark/sql/execution/basicOperators.scala | 3 +-
.../org/apache/spark/sql/execution/debug.scala | 3 +-
.../org/apache/spark/sql/execution/joins.scala | 3 +-
.../spark/sql/parquet/ParquetRelation.scala | 9 +++--
.../spark/sql/parquet/ParquetTableSupport.scala | 3 +-
.../apache/spark/sql/test/TestSQLContext.scala | 7 ++--
.../scala/org/apache/spark/sql/QueryTest.scala | 2 --
.../spark/sql/columnar/ColumnTypeSuite.scala | 3 +-
.../columnar/NullableColumnAccessorSuite.scala | 3 +-
.../columnar/NullableColumnBuilderSuite.scala | 3 +-
.../apache/spark/sql/execution/TgfSuite.scala | 3 +-
.../org/apache/spark/SparkHadoopWriter.scala | 13 ++++---
.../org/apache/spark/sql/hive/HiveContext.scala | 38 ++++++++++----------
.../spark/sql/hive/HiveMetastoreCatalog.scala | 8 ++---
.../org/apache/spark/sql/hive/HiveQl.scala | 3 +-
.../apache/spark/sql/hive/HiveStrategies.scala | 4 +--
.../spark/sql/hive/ScriptTransformation.scala | 3 +-
.../org/apache/spark/sql/hive/TableReader.scala | 3 +-
.../org/apache/spark/sql/hive/TestHive.scala | 3 +-
.../apache/spark/sql/hive/hiveOperators.scala | 3 +-
.../org/apache/spark/sql/hive/hiveUdfs.scala | 4 +--
.../hive/execution/BigDataBenchmarkSuite.scala | 10 +++---
.../hive/execution/ConcurrentHiveSuite.scala | 9 +++--
.../sql/hive/execution/HiveComparisonTest.scala | 9 +++--
.../hive/execution/HiveCompatibilitySuite.scala | 6 ++--
.../sql/hive/execution/HiveQueryFileTest.scala | 4 +--
.../sql/hive/execution/HiveQuerySuite.scala | 8 ++---
.../hive/execution/HiveResolutionSuite.scala | 7 ++--
.../sql/hive/execution/HiveSerDeSuite.scala | 4 +--
.../hive/execution/HiveTypeCoercionSuite.scala | 3 +-
.../spark/sql/hive/execution/PruningSuite.scala | 6 ++--
103 files changed, 209 insertions(+), 333 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/core/src/main/scala/org/apache/spark/Dependency.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/Dependency.scala b/core/src/main/scala/org/apache/spark/Dependency.scala
index 448f87b..3132dcf 100644
--- a/core/src/main/scala/org/apache/spark/Dependency.scala
+++ b/core/src/main/scala/org/apache/spark/Dependency.scala
@@ -44,8 +44,9 @@ abstract class NarrowDependency[T](rdd: RDD[T]) extends Dependency(rdd) {
* Represents a dependency on the output of a shuffle stage.
* @param rdd the parent RDD
* @param partitioner partitioner used to partition the shuffle output
- * @param serializer [[Serializer]] to use. If set to null, the default serializer, as specified
- * by `spark.serializer` config option, will be used.
+ * @param serializer [[org.apache.spark.serializer.Serializer Serializer]] to use. If set to null,
+ * the default serializer, as specified by `spark.serializer` config option, will
+ * be used.
*/
class ShuffleDependency[K, V](
@transient rdd: RDD[_ <: Product2[K, V]],
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
index 05b89b9..ddac553 100644
--- a/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
+++ b/core/src/main/scala/org/apache/spark/api/java/JavaRDDLike.scala
@@ -481,7 +481,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
/**
* Returns the maximum element from this RDD as defined by the specified
* Comparator[T].
- * @params comp the comparator that defines ordering
+ * @param comp the comparator that defines ordering
* @return the maximum of the RDD
* */
def max(comp: Comparator[T]): T = {
@@ -491,7 +491,7 @@ trait JavaRDDLike[T, This <: JavaRDDLike[T, This]] extends Serializable {
/**
* Returns the minimum element from this RDD as defined by the specified
* Comparator[T].
- * @params comp the comparator that defines ordering
+ * @param comp the comparator that defines ordering
* @return the minimum of the RDD
* */
def min(comp: Comparator[T]): T = {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/ScalaReflection.scala
----------------------------------------------------------------------
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 bf7318d..976dda8 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
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.expressions.Attribute
import org.apache.spark.sql.catalyst.expressions.AttributeReference
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
----------------------------------------------------------------------
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 161d28e..4ebc0e7 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
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical._
@@ -89,7 +87,8 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
/**
* Replaces [[UnresolvedAttribute]]s with concrete
- * [[expressions.AttributeReference AttributeReferences]] from a logical plan node's children.
+ * [[catalyst.expressions.AttributeReference AttributeReferences]] from a logical plan node's
+ * children.
*/
object ResolveReferences extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transformUp {
@@ -106,7 +105,7 @@ class Analyzer(catalog: Catalog, registry: FunctionRegistry, caseSensitive: Bool
}
/**
- * Replaces [[UnresolvedFunction]]s with concrete [[expressions.Expression Expressions]].
+ * Replaces [[UnresolvedFunction]]s with concrete [[catalyst.expressions.Expression Expressions]].
*/
object ResolveFunctions extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
index b77f0bb..ff66177 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Catalog.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import scala.collection.mutable
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
index eed058d..c025570 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/FunctionRegistry.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions.Expression
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
index a6ecf6e..4557d77 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project, Union}
@@ -37,8 +35,8 @@ trait HiveTypeCoercion {
StringToIntegralCasts, FunctionArgumentConversion)
/**
- * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] dataTypes
- * that are made by other rules to instances higher in the query tree.
+ * Applies any changes to [[catalyst.expressions.AttributeReference AttributeReference]] data
+ * types that are made by other rules to instances higher in the query tree.
*/
object PropagateTypes extends Rule[LogicalPlan] {
def apply(plan: LogicalPlan): LogicalPlan = plan transform {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
index 3cad3a5..a6ce908 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/MultiInstanceRelation.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
index 30c55ba..9f37ca9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* Provides a logical query plan [[Analyzer]] and supporting classes for performing analysis.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
index 04ae481..41e9bce 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/unresolved.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.{errors, trees}
import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, Expression, NamedExpression}
import org.apache.spark.sql.catalyst.plans.logical.BaseRelation
import org.apache.spark.sql.catalyst.trees.TreeNode
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
index e6255bc..67cddb3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/dsl/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import scala.language.implicitConversions
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
index d8b6994..bdeb660 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/errors/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import org.apache.spark.sql.catalyst.trees.TreeNode
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
index 3fa4148..f70e80b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/BoundAttribute.scala
@@ -15,13 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.Logging
/**
* A bound reference points to a specific slot in the input tuple, allowing the actual value
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
index 71f64ef..c26fc3d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Cast.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
index 2454a33..81fd160 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Expression.scala
@@ -15,13 +15,12 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
+import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.types.{DataType, FractionalType, IntegralType, NumericType}
-import org.apache.spark.sql.catalyst.errors.TreeNodeException
abstract class Expression extends TreeNode[Expression] {
self: Product =>
@@ -69,7 +68,7 @@ abstract class Expression extends TreeNode[Expression] {
def childrenResolved = !children.exists(!_.resolved)
/**
- * A set of helper functions that return the correct descendant of [[scala.math.Numeric]] type
+ * A set of helper functions that return the correct descendant of `scala.math.Numeric[T]` type
* and do any casting necessary of child evaluation.
*/
@inline
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
index 8c407d2..38542d3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql.catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
/**
* Converts a [[Row]] to another Row given a sequence of expression that define each column of the
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
index 0d173af..0bde621 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Rand.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.DoubleType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
index 79c91eb..31d42b9 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Row.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.NativeType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
index cc33948..f53d850 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.DataType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
index 171997b..d5d9377 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/SortOrder.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
abstract sealed class SortDirection
case object Ascending extends SortDirection
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
index 01b7a14..9828d0b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/WrapDynamic.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import scala.language.dynamics
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
index a16bb80..7303b15 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregates.scala
@@ -15,11 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.catalyst.trees
abstract class AggregateExpression extends Expression {
self: Product =>
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
index 81e4a48..fba056e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/arithmetic.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
import org.apache.spark.sql.catalyst.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
index 9ec0f6a..ab96618 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/complexTypes.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
index 9097c63..e9b491b 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/generators.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.types._
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
index 0d01312..b82a12e 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/literals.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
index 47b1241..69c8bed 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/namedExpressions.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
import org.apache.spark.sql.catalyst.types._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
index 38e3837..5a47768 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/nullFunctions.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
case class Coalesce(children: Seq[Expression]) extends Expression {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
index 76554e1..573ec05 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* A set of classes that can be used to represent trees of relational expressions. A key goal of
@@ -46,6 +45,6 @@ package catalyst
* ability to reason about which subquery produced a given attribute.
*
* ==Evaluation==
- * The result of expressions can be evaluated using the [[Evaluate]] object.
+ * The result of expressions can be evaluated using the `Expression.apply(Row)` method.
*/
package object expressions
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
index e7f3e8c..722ff51 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala
@@ -15,12 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.types.{BooleanType, StringType}
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.UnresolvedException
+import org.apache.spark.sql.catalyst.types.{BooleanType, StringType}
trait Predicate extends Expression {
self: Product =>
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
index 7584fe0..e195f2a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.apache.spark.sql.catalyst.types.BooleanType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
----------------------------------------------------------------------
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 f280769..3dd6818 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
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.Inner
@@ -125,7 +123,6 @@ object PushPredicateThroughProject extends Rule[LogicalPlan] {
grandChild))
}
- //
def replaceAlias(condition: Expression, sourceAliases: Map[Attribute, Expression]): Expression = {
condition transform {
case a: AttributeReference => sourceAliases.getOrElse(a, a)
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
index d50b963..6783366 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/QueryPlanner.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package planning
+package org.apache.spark.sql.catalyst.planning
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.TreeNode
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
index 64370ec..0a030b6 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* Contains classes for enumerating possible physical plans for a given logical query plan.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
index ff0ea90..6dd816a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/planning/patterns.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package planning
+package org.apache.spark.sql.catalyst.planning
import scala.annotation.tailrec
@@ -48,7 +46,9 @@ object FilteredOperation extends PredicateHelper {
/**
* A pattern that matches any number of project or filter operations on top of another relational
* operator. All filter operators are collected and their conditions are broken up and returned
- * together with the top project operator. [[Alias Aliases]] are in-lined/substituted if necessary.
+ * together with the top project operator.
+ * [[org.apache.spark.sql.catalyst.expressions.Alias Aliases]] are in-lined/substituted if
+ * necessary.
*/
object PhysicalOperation extends PredicateHelper {
type ReturnType = (Seq[NamedExpression], Seq[Expression], LogicalPlan)
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
index 848db24..8199a80 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/QueryPlan.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
+package org.apache.spark.sql.catalyst.plans
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
import org.apache.spark.sql.catalyst.trees.TreeNode
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
index 9f2283a..ae8d7d3 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/joinTypes.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
+package org.apache.spark.sql.catalyst.plans
sealed abstract class JoinType
case object Inner extends JoinType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
index 48ff45c..7c61678 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/BaseRelation.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
abstract class BaseRelation extends LeafNode {
self: Product =>
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/LogicalPlan.scala
----------------------------------------------------------------------
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 225dd26..5eb52d5 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
@@ -15,14 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.errors.TreeNodeException
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.QueryPlan
import org.apache.spark.sql.catalyst.types.StructType
+import org.apache.spark.sql.catalyst.trees
abstract class LogicalPlan extends QueryPlan[LogicalPlan] {
self: Product =>
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
index 5a3ea9f..d3f9d0f 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/ScriptTransformation.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
index ac7d2d6..f8fe558 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/TestRelation.scala
@@ -15,11 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.analysis
import org.apache.spark.sql.catalyst.expressions.Attribute
object LocalRelation {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
index 61481de..9d16189 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
@@ -15,12 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.plans.JoinType
import org.apache.spark.sql.catalyst.types._
case class Project(projectList: Seq[NamedExpression], child: LogicalPlan) extends UnaryNode {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
index 775e50b..7146fbd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/partitioning.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package logical
+package org.apache.spark.sql.catalyst.plans.logical
import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
index a40ab4b..42bdab4 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* A a collection of common abstractions for query plans as well as
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
index 20e2a45..8893744 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
@@ -15,10 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package plans
-package physical
+package org.apache.spark.sql.catalyst.plans.physical
import org.apache.spark.sql.catalyst.expressions.{Expression, SortOrder}
import org.apache.spark.sql.catalyst.types.IntegerType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
index c7632a6..1076537 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/Rule.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package rules
+package org.apache.spark.sql.catalyst.rules
+import org.apache.spark.sql.Logging
import org.apache.spark.sql.catalyst.trees.TreeNode
abstract class Rule[TreeType <: TreeNode[_]] extends Logging {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
index 26ab543..ca82c5d 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/rules/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* A framework for applying batches rewrite rules to trees, possibly to fixed point.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
index 89e27d8..0369129 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/TreeNode.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package trees
+package org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.errors._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
index e2da1d2..d159ecd 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/trees/package.scala
@@ -15,8 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
+
+import org.apache.spark.sql.Logger
/**
* A library for easily manipulating trees of operators. Operators that extend TreeNode are
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
index 90a9f9f..7a45d1a 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/dataTypes.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package types
+package org.apache.spark.sql.catalyst.types
import scala.reflect.runtime.universe.{typeTag, TypeTag}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
index b65a561..de24449 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/types/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
/**
* Contains a type system for attributes produced by relations, including complex types like
* structs, arrays and maps.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
index 52adea2..a001d95 100644
--- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
+++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/util/package.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import java.io.{PrintWriter, ByteArrayOutputStream, FileInputStream, File}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
index fb25e1c..46b2250 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/DistributionSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
+package org.apache.spark.sql.catalyst
import org.scalatest.FunSuite
@@ -172,4 +171,4 @@ class DistributionSuite extends FunSuite {
AllTuples,
false)
}
-}
\ No newline at end of file
+}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
index 78ec48b..4c31358 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/AnalysisSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.scalatest.FunSuite
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
index b85b72a..b9e0f8e 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercionSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package analysis
+package org.apache.spark.sql.catalyst.analysis
import org.scalatest.FunSuite
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
index c8fd581..94894ad 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package expressions
+package org.apache.spark.sql.catalyst.expressions
import org.scalatest.FunSuite
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/ConstantFoldingSuite.scala
----------------------------------------------------------------------
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 53f760f..2ab14f4 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
@@ -15,11 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
-import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.plans.logical.{LocalRelation, LogicalPlan}
import org.apache.spark.sql.catalyst.rules.RuleExecutor
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
index ae1b2b1..ef47850 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/FilterPushdownSuite.scala
@@ -15,12 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
-
-import org.apache.spark.sql.catalyst.analysis._
+import org.apache.spark.sql.catalyst.analysis
+import org.apache.spark.sql.catalyst.analysis.EliminateAnalysisOperators
import org.apache.spark.sql.catalyst.plans.logical._
import org.apache.spark.sql.catalyst.rules._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
index 0c6e4d5..89982d5 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/OptimizerTest.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package optimizer
+package org.apache.spark.sql.catalyst.optimizer
import org.scalatest.FunSuite
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
index 738cfa8..4b2d455 100644
--- a/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
+++ b/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/trees/RuleExecutorSuite.scala
@@ -15,9 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package catalyst
-package trees
+package org.apache.spark.sql.catalyst.trees
import org.scalatest.FunSuite
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
index ddbeba6..e0c98ec 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnAccessor.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import java.nio.{ByteOrder, ByteBuffer}
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
index 6bd1841..3e622ad 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnBuilder.scala
@@ -15,12 +15,13 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
-import java.nio.{ByteOrder, ByteBuffer}
+import java.nio.{ByteBuffer, ByteOrder}
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.types._
+import org.apache.spark.sql.columnar.ColumnBuilder._
import org.apache.spark.sql.execution.SparkSqlSerializer
private[sql] trait ColumnBuilder {
@@ -35,7 +36,6 @@ private[sql] trait ColumnBuilder {
}
private[sql] abstract class BasicColumnBuilder[T <: DataType, JvmType] extends ColumnBuilder {
- import ColumnBuilder._
private var columnName: String = _
protected var buffer: ByteBuffer = _
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
index 3b759a5..a452b86 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/ColumnType.scala
@@ -14,8 +14,8 @@
* See the License for the specific language governing permissions and
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+
+package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
index 1661c3f..048d1f0 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/NullableColumnBuilder.scala
@@ -15,10 +15,11 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
-import java.nio.{ByteOrder, ByteBuffer}
+import java.nio.{ByteBuffer, ByteOrder}
+
+import org.apache.spark.sql.Row
/**
* Builds a nullable column. The byte buffer of a nullable column contains:
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
index c7efd30..f853759 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/columnar/inMemoryColumnarOperators.scala
@@ -15,18 +15,18 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import org.apache.spark.sql.catalyst.expressions.{GenericMutableRow, Attribute}
import org.apache.spark.sql.execution.{SparkPlan, LeafNode}
+import org.apache.spark.sql.Row
+
+/* Implicit conversions */
+import org.apache.spark.sql.columnar.ColumnType._
private[sql] case class InMemoryColumnarTableScan(attributes: Seq[Attribute], child: SparkPlan)
extends LeafNode {
- // For implicit conversion from `DataType` to `ColumnType`
- import ColumnType._
-
override def output: Seq[Attribute] = attributes
lazy val cachedColumnBuffers = {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
index 65d77e3..869673b 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Exchange.scala
@@ -15,16 +15,16 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
+import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf}
import org.apache.spark.rdd.ShuffledRDD
+import org.apache.spark.sql.Row
import org.apache.spark.sql.catalyst.errors.attachTree
import org.apache.spark.sql.catalyst.expressions.{MutableProjection, RowOrdering}
import org.apache.spark.sql.catalyst.plans.physical._
import org.apache.spark.sql.catalyst.rules.Rule
import org.apache.spark.util.MutablePair
-import org.apache.spark.{HashPartitioner, RangePartitioner, SparkConf}
case class Exchange(newPartitioning: Partitioning, child: SparkPlan) extends UnaryNode {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
index 7e50fda..e902e6c 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/Generate.scala
@@ -15,10 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{Generator, JoinedRow, Literal, Projection}
/**
* Applies a [[catalyst.expressions.Generator Generator]] to a stream of input rows, combining the
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
index 7ce8608..16806c6 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/QueryExecutionException.scala
@@ -15,7 +15,6 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
class QueryExecutionException(message: String) extends Exception(message)
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
index 5626181..acb1ee8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
@@ -15,16 +15,15 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import org.apache.spark.rdd.RDD
-
+import org.apache.spark.sql.{Logging, Row}
+import org.apache.spark.sql.catalyst.trees
import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
-import org.apache.spark.sql.catalyst.plans.QueryPlan
-import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.catalyst.expressions.GenericRow
+import org.apache.spark.sql.catalyst.plans.{QueryPlan, logical}
import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.trees
abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
self: Product =>
@@ -47,7 +46,7 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with Logging {
def executeCollect(): Array[Row] = execute().collect()
protected def buildRow(values: Seq[Any]): Row =
- new catalyst.expressions.GenericRow(values.toArray)
+ new GenericRow(values.toArray)
}
/**
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
index ad7cd58..1c3196a 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkSqlSerializer.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import java.nio.ByteBuffer
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
index 8a39ded..86f9d3e 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/SparkStrategies.scala
@@ -15,9 +15,9 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
+import org.apache.spark.sql.{SQLContext, execution}
import org.apache.spark.sql.catalyst.expressions._
import org.apache.spark.sql.catalyst.planning._
import org.apache.spark.sql.catalyst.plans._
@@ -175,7 +175,7 @@ abstract class SparkStrategies extends QueryPlanner[SparkPlan] {
InsertIntoParquetTable(relation, planLater(child))(sparkContext) :: Nil
case logical.InsertIntoTable(table: ParquetRelation, partition, child, overwrite) =>
InsertIntoParquetTable(table, planLater(child))(sparkContext) :: Nil
- case PhysicalOperation(projectList, filters, relation: parquet.ParquetRelation) =>
+ case PhysicalOperation(projectList, filters, relation: ParquetRelation) =>
// TODO: Should be pushing down filters as well.
pruneFilterProject(
projectList,
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
index 14e5ab6..8515a18 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregates.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import org.apache.spark.SparkContext
import org.apache.spark.sql.catalyst.errors._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
index e4f918b..65cb8f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/basicOperators.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import scala.reflect.runtime.universe.TypeTag
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
index db259b4..40982f1 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/debug.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
object DebugQuery {
def apply(plan: SparkPlan): SparkPlan = {
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
index a6e3892..f0d2114 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/joins.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import scala.collection.mutable
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
index 011aaf7..2b825f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetRelation.scala
@@ -82,11 +82,10 @@ object ParquetRelation {
type RowType = org.apache.spark.sql.catalyst.expressions.GenericMutableRow
/**
- * Creates a new ParquetRelation and underlying Parquetfile for the given
- * LogicalPlan. Note that this is used inside [[SparkStrategies]] to
- * create a resolved relation as a data sink for writing to a Parquetfile.
- * The relation is empty but is initialized with ParquetMetadata and
- * can be inserted into.
+ * Creates a new ParquetRelation and underlying Parquetfile for the given LogicalPlan. Note that
+ * this is used inside [[org.apache.spark.sql.execution.SparkStrategies SparkStrategies]] to
+ * create a resolved relation as a data sink for writing to a Parquetfile. The relation is empty
+ * but is initialized with ParquetMetadata and can be inserted into.
*
* @param pathString The directory the Parquetfile will be stored in.
* @param child The child node that will be used for extracting the schema.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
index 91b4848..c21e400 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTableSupport.scala
@@ -135,8 +135,7 @@ object RowWriteSupport {
}
/**
- * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record
- * to a [[org.apache.spark.sql.catalyst.expressions.Row]] object.
+ * A `parquet.io.api.GroupConverter` that is able to convert a Parquet record to a `Row` object.
*
* @param schema The corresponding Catalyst schema in the form of a list of attributes.
*/
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
index ca56c44..f2389f8 100644
--- a/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
+++ b/sql/core/src/main/scala/org/apache/spark/sql/test/TestSQLContext.scala
@@ -15,9 +15,10 @@
* limitations under the License.
*/
-package org.apache.spark
-package sql
-package test
+package org.apache.spark.sql.test
+
+import org.apache.spark.{SparkConf, SparkContext}
+import org.apache.spark.sql.SQLContext
/** A SQLContext that can be used for local testing. */
object TestSQLContext
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
index 5c8cb08..d719ceb 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/QueryTest.scala
@@ -22,8 +22,6 @@ import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.plans._
import org.apache.spark.sql.catalyst.util._
-/* Implicits */
-
class QueryTest extends FunSuite {
/**
* Runs the plan and makes sure the answer matches the expected result.
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
index 1997d86..2d431af 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/ColumnTypeSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import java.nio.ByteBuffer
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
index 279607c..d413d48 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnAccessorSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import org.scalatest.FunSuite
import org.apache.spark.sql.catalyst.types.DataType
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
index 3354da3..5222a47 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/columnar/NullableColumnBuilderSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package columnar
+package org.apache.spark.sql.columnar
import org.scalatest.FunSuite
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
----------------------------------------------------------------------
diff --git a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
index 93b2a30..ca5c8b8 100644
--- a/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
+++ b/sql/core/src/test/scala/org/apache/spark/sql/execution/TgfSuite.scala
@@ -15,8 +15,7 @@
* limitations under the License.
*/
-package org.apache.spark.sql
-package execution
+package org.apache.spark.sql.execution
import org.apache.spark.sql.QueryTest
import org.apache.spark.sql.catalyst.expressions._
http://git-wip-us.apache.org/repos/asf/spark/blob/345825d9/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
----------------------------------------------------------------------
diff --git a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
index d96c2f7..7219c03 100644
--- a/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
+++ b/sql/hive/src/main/scala/org/apache/spark/SparkHadoopWriter.scala
@@ -113,11 +113,10 @@ class SparkHiveHadoopWriter(
cmtr.commitTask(taCtxt)
logInfo (taID + ": Committed")
} catch {
- case e: IOException => {
+ case e: IOException =>
logError("Error committing the output of task: " + taID.value, e)
cmtr.abortTask(taCtxt)
throw e
- }
}
} else {
logWarning ("No need to commit output of task: " + taID.value)
@@ -161,12 +160,12 @@ class SparkHiveHadoopWriter(
taskContext
}
- private def setIDs(jobid: Int, splitid: Int, attemptid: Int) {
- jobID = jobid
- splitID = splitid
- attemptID = attemptid
+ private def setIDs(jobId: Int, splitId: Int, attemptId: Int) {
+ jobID = jobId
+ splitID = splitId
+ attemptID = attemptId
- jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobid))
+ jID = new SerializableWritable[JobID](SparkHadoopWriter.createJobID(now, jobId))
taID = new SerializableWritable[TaskAttemptID](
new TaskAttemptID(new TaskID(jID.value, true, splitID), attemptID))
}