aboutsummaryrefslogtreecommitdiff
path: root/sql
Commit message (Collapse)AuthorAgeFilesLines
* [SPARK-2783][SQL] Basic support for analyze in HiveContextYin Huai2014-08-033-2/+136
| | | | | | | | | | | | | | | | | | | | JIRA: https://issues.apache.org/jira/browse/SPARK-2783 Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1741 from yhuai/analyzeTable and squashes the following commits: 7bb5f02 [Yin Huai] Use sql instead of hql. 4d09325 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable e3ebcd4 [Yin Huai] Renaming. c170f4e [Yin Huai] Do not use getContentSummary. 62393b6 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable db233a6 [Yin Huai] Trying to debug jenkins... fee84f0 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable f0501f3 [Yin Huai] Fix compilation error. 24ad391 [Yin Huai] Merge remote-tracking branch 'upstream/master' into analyzeTable 8918140 [Yin Huai] Wording. 23df227 [Yin Huai] Add a simple analyze method to get the size of a table and update the "totalSize" property of this table in the Hive metastore.
* [SPARK-2814][SQL] HiveThriftServer2 throws NPE when executing native commandsCheng Lian2014-08-031-4/+3
| | | | | | | | | | JIRA issue: [SPARK-2814](https://issues.apache.org/jira/browse/SPARK-2814) Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1753 from liancheng/spark-2814 and squashes the following commits: c74a3b2 [Cheng Lian] Fixed SPARK-2814
* [SPARK-2784][SQL] Deprecate hql() method in favor of a config option, ↵Michael Armbrust2014-08-0317-106/+168
| | | | | | | | | | | | | | | | | | | | 'spark.sql.dialect' Many users have reported being confused by the distinction between the `sql` and `hql` methods. Specifically, many users think that `sql(...)` cannot be used to read hive tables. In this PR I introduce a new configuration option `spark.sql.dialect` that picks which dialect with be used for parsing. For SQLContext this must be set to `sql`. In `HiveContext` it defaults to `hiveql` but can also be set to `sql`. The `hql` and `hiveql` methods continue to act the same but are now marked as deprecated. **This is a possibly breaking change for some users unless they set the dialect manually, though this is unlikely.** For example: `hiveContex.sql("SELECT 1")` will now throw a parsing exception by default. Author: Michael Armbrust <michael@databricks.com> Closes #1746 from marmbrus/sqlLanguageConf and squashes the following commits: ad375cc [Michael Armbrust] Merge remote-tracking branch 'apache/master' into sqlLanguageConf 20c43f8 [Michael Armbrust] override function instead of just setting the value 7e4ae93 [Michael Armbrust] Deprecate hql() method in favor of a config option, 'spark.sql.dialect'
* [SPARK-2739][SQL] Rename registerAsTable to registerTempTableMichael Armbrust2014-08-0219-75/+78
| | | | | | | | | | | | | There have been user complaints that the difference between `registerAsTable` and `saveAsTable` is too subtle. This PR addresses this by renaming `registerAsTable` to `registerTempTable`, which more clearly reflects what is happening. `registerAsTable` remains, but will cause a deprecation warning. Author: Michael Armbrust <michael@databricks.com> Closes #1743 from marmbrus/registerTempTable and squashes the following commits: d031348 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into registerTempTable 4dff086 [Michael Armbrust] Fix .java files too 89a2f12 [Michael Armbrust] Merge remote-tracking branch 'apache/master' into registerTempTable 0b7b71e [Michael Armbrust] Rename registerAsTable to registerTempTable
* [SPARK-2729][SQL] Added test case for SPARK-2729Cheng Lian2014-08-022-2/+22
| | | | | | | | | | This is a follow up of #1636. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1738 from liancheng/test-for-spark-2729 and squashes the following commits: b13692a [Cheng Lian] Added test case for SPARK-2729
* [SPARK-2785][SQL] Remove assertions that throw when users try unsupported ↵Michael Armbrust2014-08-021-4/+9
| | | | | | | | | | Hive commands. Author: Michael Armbrust <michael@databricks.com> Closes #1742 from marmbrus/asserts and squashes the following commits: 5182d54 [Michael Armbrust] Remove assertions that throw when users try unsupported Hive commands.
* [SPARK-2097][SQL] UDF SupportMichael Armbrust2014-08-0237-18/+1823
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch adds the ability to register lambda functions written in Python, Java or Scala as UDFs for use in SQL or HiveQL. Scala: ```scala registerFunction("strLenScala", (_: String).length) sql("SELECT strLenScala('test')") ``` Python: ```python sqlCtx.registerFunction("strLenPython", lambda x: len(x), IntegerType()) sqlCtx.sql("SELECT strLenPython('test')") ``` Java: ```java sqlContext.registerFunction("stringLengthJava", new UDF1<String, Integer>() { Override public Integer call(String str) throws Exception { return str.length(); } }, DataType.IntegerType); sqlContext.sql("SELECT stringLengthJava('test')"); ``` Author: Michael Armbrust <michael@databricks.com> Closes #1063 from marmbrus/udfs and squashes the following commits: 9eda0fe [Michael Armbrust] newline 747c05e [Michael Armbrust] Add some scala UDF tests. d92727d [Michael Armbrust] Merge remote-tracking branch 'apache/master' into udfs 005d684 [Michael Armbrust] Fix naming and formatting. d14dac8 [Michael Armbrust] Fix last line of autogened java files. 8135c48 [Michael Armbrust] Move UDF unit tests to pyspark. 40b0ffd [Michael Armbrust] Merge remote-tracking branch 'apache/master' into udfs 6a36890 [Michael Armbrust] Switch logging so that SQLContext can be serializable. 7a83101 [Michael Armbrust] Drop toString 795fd15 [Michael Armbrust] Try to avoid capturing SQLContext. e54fb45 [Michael Armbrust] Docs and tests. 437cbe3 [Michael Armbrust] Update use of dataTypes, fix some python tests, address review comments. 01517d6 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udfs 8e6c932 [Michael Armbrust] WIP 3f96a52 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into udfs 6237c8d [Michael Armbrust] WIP 2766f0b [Michael Armbrust] Move udfs support to SQL from hive. Add support for Java UDFs. 0f7d50c [Michael Armbrust] Draft of native Spark SQL UDFs for Scala and Python.
* SPARK-2804: Remove scalalogging-slf4j dependencyGuoQiang Li2014-08-0229-79/+76
| | | | | | | | | | This also Closes #1701. Author: GuoQiang Li <witgo@qq.com> Closes #1208 from witgo/SPARK-1470 and squashes the following commits: 422646b [GuoQiang Li] Remove scalalogging-slf4j dependency
* [SQL] Set outputPartitioning of BroadcastHashJoin correctly.Yin Huai2014-08-021-2/+1
| | | | | | | | | | | | | | | | | | | | | | | | I think we will not generate the plan triggering this bug at this moment. But, let me explain it... Right now, we are using `left.outputPartitioning` as the `outputPartitioning` of a `BroadcastHashJoin`. We may have a wrong physical plan for cases like... ```sql SELECT l.key, count(*) FROM (SELECT key, count(*) as cnt FROM src GROUP BY key) l // This is buildPlan JOIN r // This is the streamedPlan ON (l.cnt = r.value) GROUP BY l.key ``` Let's say we have a `BroadcastHashJoin` on `l` and `r`. For this case, we will pick `l`'s `outputPartitioning` for the `outputPartitioning`of the `BroadcastHashJoin` on `l` and `r`. Also, because the last `GROUP BY` is using `l.key` as the key, we will not introduce an `Exchange` for this aggregation. However, `r`'s outputPartitioning may not match the required distribution of the last `GROUP BY` and we fail to group data correctly. JIRA is being reindexed. I will create a JIRA ticket once it is back online. Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1735 from yhuai/BroadcastHashJoin and squashes the following commits: 96d9cb3 [Yin Huai] Set outputPartitioning correctly.
* [SPARK-1812] sql/catalyst - Provide explicit type informationAnand Avati2014-08-021-1/+1
| | | | | | | | | | | | | | | | | | | | | | | | | | | | For Scala 2.11 compatibility. Without the explicit type specification, withNullability return type is inferred to be Attribute, and thus calling at() on the returned object fails in these tests: [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:370: value at is not a [ERROR] val c4_notNull = 'a.boolean.notNull.at(3) [ERROR] ^ [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:371: value at is not a [ERROR] val c5_notNull = 'a.boolean.notNull.at(4) [ERROR] ^ [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:372: value at is not a [ERROR] val c6_notNull = 'a.boolean.notNull.at(5) [ERROR] ^ [ERROR] /Users/avati/work/spark/sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/expressions/ExpressionEvaluationSuite.scala:558: value at is not a [ERROR] val s_notNull = 'a.string.notNull.at(0) Signed-off-by: Anand Avati <avatiredhat.com> Author: Anand Avati <avati@redhat.com> Closes #1709 from avati/SPARK-1812-notnull and squashes the following commits: 0470eb3 [Anand Avati] SPARK-1812: sql/catalyst - Provide explicit type information
* Revert "[SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of ↵Patrick Wendell2014-08-0129-76/+79
| | | | | | the directly sfl4j api" This reverts commit adc8303294e26efb4ed15e5f5ba1062f7988625d.
* [SPARK-1470][SPARK-1842] Use the scala-logging wrapper instead of the ↵GuoQiang Li2014-08-0129-79/+76
| | | | | | | | | | | directly sfl4j api Author: GuoQiang Li <witgo@qq.com> Closes #1369 from witgo/SPARK-1470_new and squashes the following commits: 66a1641 [GuoQiang Li] IncompatibleResultTypeProblem 73a89ba [GuoQiang Li] Use the scala-logging wrapper instead of the directly sfl4j api.
* [SPARK-2212][SQL] Hash Outer Join (follow-up bug fix).Yin Huai2014-08-013-2/+114
| | | | | | | | | | We need to carefully set the ouputPartitioning of the HashOuterJoin Operator. Otherwise, we may not correctly handle nulls. Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1721 from yhuai/SPARK-2212-BugFix and squashes the following commits: ed5eef7 [Yin Huai] Correctly choosing outputPartitioning for the HashOuterJoin operator.
* [SPARK-2010] [PySpark] [SQL] support nested structure in SchemaRDDDavies Liu2014-08-012-77/+28
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Convert Row in JavaSchemaRDD into Array[Any] and unpickle them as tuple in Python, then convert them into namedtuple, so use can access fields just like attributes. This will let nested structure can be accessed as object, also it will reduce the size of serialized data and better performance. root |-- field1: integer (nullable = true) |-- field2: string (nullable = true) |-- field3: struct (nullable = true) | |-- field4: integer (nullable = true) | |-- field5: array (nullable = true) | | |-- element: integer (containsNull = false) |-- field6: array (nullable = true) | |-- element: struct (containsNull = false) | | |-- field7: string (nullable = true) Then we can access them by row.field3.field5[0] or row.field6[5].field7 It also will infer the schema in Python, convert Row/dict/namedtuple/objects into tuple before serialization, then call applySchema in JVM. During inferSchema(), the top level of dict in row will be StructType, but any nested dictionary will be MapType. You can use pyspark.sql.Row to convert unnamed structure into Row object, make the RDD can be inferable. Such as: ctx.inferSchema(rdd.map(lambda x: Row(a=x[0], b=x[1])) Or you could use Row to create a class just like namedtuple, for example: Person = Row("name", "age") ctx.inferSchema(rdd.map(lambda x: Person(*x))) Also, you can call applySchema to apply an schema to a RDD of tuple/list and turn it into a SchemaRDD. The `schema` should be StructType, see the API docs for details. schema = StructType([StructField("name, StringType, True), StructType("age", IntegerType, True)]) ctx.applySchema(rdd, schema) PS: In order to use namedtuple to inferSchema, you should make namedtuple picklable. Author: Davies Liu <davies.liu@gmail.com> Closes #1598 from davies/nested and squashes the following commits: f1d15b6 [Davies Liu] verify schema with the first few rows 8852aaf [Davies Liu] check type of schema abe9e6e [Davies Liu] address comments 61b2292 [Davies Liu] add @deprecated to pythonToJavaMap 1e5b801 [Davies Liu] improve cache of classes 51aa135 [Davies Liu] use Row to infer schema e9c0d5c [Davies Liu] remove string typed schema 353a3f2 [Davies Liu] fix code style 63de8f8 [Davies Liu] fix typo c79ca67 [Davies Liu] fix serialization of nested data 6b258b5 [Davies Liu] fix pep8 9d8447c [Davies Liu] apply schema provided by string of names f5df97f [Davies Liu] refactor, address comments 9d9af55 [Davies Liu] use arrry to applySchema and infer schema in Python 84679b3 [Davies Liu] Merge branch 'master' of github.com:apache/spark into nested 0eaaf56 [Davies Liu] fix doc tests b3559b4 [Davies Liu] use generated Row instead of namedtuple c4ddc30 [Davies Liu] fix conflict between name of fields and variables 7f6f251 [Davies Liu] address all comments d69d397 [Davies Liu] refactor 2cc2d45 [Davies Liu] refactor 182fb46 [Davies Liu] refactor bc6e9e1 [Davies Liu] switch to new Schema API 547bf3e [Davies Liu] Merge branch 'master' into nested a435b5a [Davies Liu] add docs and code refactor 2c8debc [Davies Liu] Merge branch 'master' into nested 644665a [Davies Liu] use tuple and namedtuple for schemardd
* [SPARK-2767] [SQL] SparkSQL CLI doens't output error message if query failed.Cheng Hao2014-08-013-6/+8
| | | | | | | | | Author: Cheng Hao <hao.cheng@intel.com> Closes #1686 from chenghao-intel/spark_sql_cli and squashes the following commits: eb664cc [Cheng Hao] Output detailed failure message in console 93b0382 [Cheng Hao] Fix Bug of no output in cli if exception thrown internally
* [SPARK-2729] [SQL] Forgot to match Timestamp type in ColumnBuilderchutium2014-08-013-2/+3
| | | | | | | | | | | | just a match forgot, found after SPARK-2710 , TimestampType can be used by a SchemaRDD generated from JDBC ResultSet Author: chutium <teng.qiu@gmail.com> Closes #1636 from chutium/SPARK-2729 and squashes the following commits: 71af77a [chutium] [SPARK-2729] [SQL] added Timestamp in NullableColumnAccessorSuite 39cf9f8 [chutium] [SPARK-2729] add Timestamp Type into ColumnBuilder TestSuite, ref. #1636 ab6ff97 [chutium] [SPARK-2729] Forgot to match Timestamp type in ColumnBuilder
* [SQL][SPARK-2212]Hash Outer JoinCheng Hao2014-08-013-6/+319
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This patch is to support the hash based outer join. Currently, outer join for big relations are resort to `BoradcastNestedLoopJoin`, which is super slow. This PR will create 2 hash tables for both relations in the same partition, which greatly reduce the table scans. Here is the testing code that I used: ``` package org.apache.spark.sql.hive import org.apache.spark.SparkContext import org.apache.spark.SparkConf import org.apache.spark.sql._ case class Record(key: String, value: String) object JoinTablePrepare extends App { import TestHive2._ val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"${i % 828193}", s"val_$i"))) runSqlHive("SHOW TABLES") runSqlHive("DROP TABLE if exists a") runSqlHive("DROP TABLE if exists b") runSqlHive("DROP TABLE if exists result") rdd.registerAsTable("records") runSqlHive("""CREATE TABLE a (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) runSqlHive("""CREATE TABLE b (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) runSqlHive("""CREATE TABLE result (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) hql(s"""from records | insert into table a | select key, value """.stripMargin) hql(s"""from records | insert into table b select key + 100000, value """.stripMargin) } object JoinTablePerformanceTest extends App { import TestHive2._ hql("SHOW TABLES") hql("set spark.sql.shuffle.partitions=20") val leftOuterJoin = "insert overwrite table result select a.key, b.value from a left outer join b on a.key=b.key" val rightOuterJoin = "insert overwrite table result select a.key, b.value from a right outer join b on a.key=b.key" val fullOuterJoin = "insert overwrite table result select a.key, b.value from a full outer join b on a.key=b.key" val results = ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("LeftOuterJoin", benchmark(leftOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("RightOuterJoin", benchmark(rightOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: ("FullOuterJoin", benchmark(fullOuterJoin)) :: Nil val explains = hql(s"explain $leftOuterJoin").collect ++ hql(s"explain $rightOuterJoin").collect ++ hql(s"explain $fullOuterJoin").collect println(explains.mkString(",\n")) results.foreach { case (prompt, result) => { println(s"$prompt: took ${result._1} ms (${result._2} records)") } } def benchmark(cmd: String) = { val begin = System.currentTimeMillis() val result = hql(cmd) val end = System.currentTimeMillis() val count = hql("select count(1) from result").collect.mkString("") ((end - begin), count) } } ``` And the result as shown below: ``` [Physical execution plan:], [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true], [ Project [key#95,value#98]], [ HashOuterJoin [key#95], [key#97], LeftOuter, None], [ Exchange (HashPartitioning [key#95], 20)], [ HiveTableScan [key#95], (MetastoreRelation default, a, None), None], [ Exchange (HashPartitioning [key#97], 20)], [ HiveTableScan [key#97,value#98], (MetastoreRelation default, b, None), None], [Physical execution plan:], [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true], [ Project [key#102,value#105]], [ HashOuterJoin [key#102], [key#104], RightOuter, None], [ Exchange (HashPartitioning [key#102], 20)], [ HiveTableScan [key#102], (MetastoreRelation default, a, None), None], [ Exchange (HashPartitioning [key#104], 20)], [ HiveTableScan [key#104,value#105], (MetastoreRelation default, b, None), None], [Physical execution plan:], [InsertIntoHiveTable (MetastoreRelation default, result, None), Map(), true], [ Project [key#109,value#112]], [ HashOuterJoin [key#109], [key#111], FullOuter, None], [ Exchange (HashPartitioning [key#109], 20)], [ HiveTableScan [key#109], (MetastoreRelation default, a, None), None], [ Exchange (HashPartitioning [key#111], 20)], [ HiveTableScan [key#111,value#112], (MetastoreRelation default, b, None), None] LeftOuterJoin: took 16072 ms ([3000000] records) LeftOuterJoin: took 14394 ms ([3000000] records) RightOuterJoin: took 14802 ms ([3000000] records) RightOuterJoin: took 14747 ms ([3000000] records) FullOuterJoin: took 17715 ms ([6000000] records) FullOuterJoin: took 17629 ms ([6000000] records) ``` Without this PR, the benchmark will run seems never end. Author: Cheng Hao <hao.cheng@intel.com> Closes #1147 from chenghao-intel/hash_based_outer_join and squashes the following commits: 65c599e [Cheng Hao] Fix issues with the community comments 72b1394 [Cheng Hao] Fix bug of stale value in joinedRow 55baef7 [Cheng Hao] Add HashOuterJoin
* [SPARK-2179][SQL] A minor refactoring Java data type APIs (2179 follow-up).Yin Huai2014-08-0124-118/+108
| | | | | | | | | | | It is a follow-up PR of SPARK-2179 (https://issues.apache.org/jira/browse/SPARK-2179). It makes package names of data type APIs more consistent across languages (Scala: `org.apache.spark.sql`, Java: `org.apache.spark.sql.api.java`, Python: `pyspark.sql`). Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1712 from yhuai/javaDataType and squashes the following commits: 62eb705 [Yin Huai] Move package-info. add4bcb [Yin Huai] Make the package names of data type classes consistent across languages by moving all Java data type classes to package sql.api.java.
* SPARK-2766: ScalaReflectionSuite throw an llegalArgumentException in JDK 6GuoQiang Li2014-07-311-1/+1
| | | | | | | | Author: GuoQiang Li <witgo@qq.com> Closes #1683 from witgo/SPARK-2766 and squashes the following commits: d0db00c [GuoQiang Li] ScalaReflectionSuite throw an llegalArgumentException in JDK 6
* [SPARK-2779] [SQL] asInstanceOf[Map[...]] should use scala.collection.Map ↵Yin Huai2014-07-314-0/+24
| | | | | | | | | | | | | | instead of scala.collection.immutable.Map Since we let users create Rows. It makes sense to accept mutable Maps as values of MapType columns. JIRA: https://issues.apache.org/jira/browse/SPARK-2779 Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1705 from yhuai/SPARK-2779 and squashes the following commits: 00d72fd [Yin Huai] Use scala.collection.Map.
* [SPARK-2531 & SPARK-2436] [SQL] Optimize the BuildSide when planning ↵Zongheng Yang2014-07-312-28/+55
| | | | | | | | | | | | | | | | | | BroadcastNestedLoopJoin. This PR resolves the following two tickets: - [SPARK-2531](https://issues.apache.org/jira/browse/SPARK-2531): BNLJ currently assumes the build side is the right relation. This patch refactors some of its logic to take into account a BuildSide properly. - [SPARK-2436](https://issues.apache.org/jira/browse/SPARK-2436): building on top of the above, we simply use the physical size statistics (if available) of both relations, and make the smaller relation the build side in the planner. Author: Zongheng Yang <zongheng.y@gmail.com> Closes #1448 from concretevitamin/bnlj-buildSide and squashes the following commits: 1780351 [Zongheng Yang] Use size estimation to decide optimal build side of BNLJ. 68e6c5b [Zongheng Yang] Consolidate two adjacent pattern matchings. 96d312a [Zongheng Yang] Use a while loop instead of collection methods chaining. 4bc525e [Zongheng Yang] Make BroadcastNestedLoopJoin take a BuildSide.
* [SPARK-2523] [SQL] Hadoop table scan bug fixing (fix failing Jenkins maven test)Yin Huai2014-07-316-26/+19
| | | | | | | | | | This PR tries to resolve the broken Jenkins maven test issue introduced by #1439. Now, we create a single query test to run both the setup work and the test query. Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1669 from yhuai/SPARK-2523-fixTest and squashes the following commits: 358af1a [Yin Huai] Make partition_based_table_scan_with_different_serde run atomically.
* [SPARK-2397][SQL] Deprecate LocalHiveContextMichael Armbrust2014-07-312-5/+17
| | | | | | | | | | | | | LocalHiveContext is redundant with HiveContext. The only difference is it creates `./metastore` instead of `./metastore_db`. Author: Michael Armbrust <michael@databricks.com> Closes #1641 from marmbrus/localHiveContext and squashes the following commits: e5ec497 [Michael Armbrust] Add deprecation version 626e056 [Michael Armbrust] Don't remove from imports yet 905cc5f [Michael Armbrust] Merge remote-tracking branch 'apache/master' into localHiveContext 1c2727e [Michael Armbrust] Deprecate LocalHiveContext
* [SPARK-2743][SQL] Resolve original attributes in ParquetTableScanMichael Armbrust2014-07-313-17/+28
| | | | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #1647 from marmbrus/parquetCase and squashes the following commits: a1799b7 [Michael Armbrust] move comment 2a2a68b [Michael Armbrust] Merge remote-tracking branch 'apache/master' into parquetCase bb35d5b [Michael Armbrust] Fix test case that produced an invalid plan. e6870bf [Michael Armbrust] Better error message. 539a2e1 [Michael Armbrust] Resolve original attributes in ParquetTableScan
* SPARK-2045 Sort-based shuffleMatei Zaharia2014-07-302-4/+4
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | This adds a new ShuffleManager based on sorting, as described in https://issues.apache.org/jira/browse/SPARK-2045. The bulk of the code is in an ExternalSorter class that is similar to ExternalAppendOnlyMap, but sorts key-value pairs by partition ID and can be used to create a single sorted file with a map task's output. (Longer-term I think this can take on the remaining functionality in ExternalAppendOnlyMap and replace it so we don't have code duplication.) The main TODOs still left are: - [x] enabling ExternalSorter to merge across spilled files - [x] with an Ordering - [x] without an Ordering, using the keys' hash codes - [x] adding more tests (e.g. a version of our shuffle suite that runs on this) - [x] rebasing on top of the size-tracking refactoring in #1165 when that is merged - [x] disabling spilling if spark.shuffle.spill is set to false Despite this though, this seems to work pretty well (running successfully in cases where the hash shuffle would OOM, such as 1000 reduce tasks on executors with only 1G memory), and it seems to be comparable in speed or faster than hash-based shuffle (it will create much fewer files for the OS to keep track of). So I'm posting it to get some early feedback. After these TODOs are done, I'd also like to enable ExternalSorter to sort data within each partition by a key as well, which will allow us to use it to implement external spilling in reduce tasks in `sortByKey`. Author: Matei Zaharia <matei@databricks.com> Closes #1499 from mateiz/sort-based-shuffle and squashes the following commits: bd841f9 [Matei Zaharia] Various review comments d1c137fd [Matei Zaharia] Various review comments a611159 [Matei Zaharia] Compile fixes due to rebase 62c56c8 [Matei Zaharia] Fix ShuffledRDD sometimes not returning Tuple2s. f617432 [Matei Zaharia] Fix a failing test (seems to be due to change in SizeTracker logic) 9464d5f [Matei Zaharia] Simplify code and fix conflicts after latest rebase 0174149 [Matei Zaharia] Add cleanup behavior and cleanup tests for sort-based shuffle eb4ee0d [Matei Zaharia] Remove customizable element type in ShuffledRDD fa2e8db [Matei Zaharia] Allow nextBatchStream to be called after we're done looking at all streams a34b352 [Matei Zaharia] Fix tracking of indices within a partition in SpillReader, and add test 03e1006 [Matei Zaharia] Add a SortShuffleSuite that runs ShuffleSuite with sort-based shuffle 3c7ff1f [Matei Zaharia] Obey the spark.shuffle.spill setting in ExternalSorter ad65fbd [Matei Zaharia] Rebase on top of Aaron's Sorter change, and use Sorter in our buffer 44d2a93 [Matei Zaharia] Use estimateSize instead of atGrowThreshold to test collection sizes 5686f71 [Matei Zaharia] Optimize merging phase for in-memory only data: 5461cbb [Matei Zaharia] Review comments and more tests (e.g. tests with 1 element per partition) e9ad356 [Matei Zaharia] Update ContextCleanerSuite to make sure shuffle cleanup tests use hash shuffle (since they were written for it) c72362a [Matei Zaharia] Added bug fix and test for when iterators are empty de1fb40 [Matei Zaharia] Make trait SizeTrackingCollection private[spark] 4988d16 [Matei Zaharia] tweak c1b7572 [Matei Zaharia] Small optimization ba7db7f [Matei Zaharia] Handle null keys in hash-based comparator, and add tests for collisions ef4e397 [Matei Zaharia] Support for partial aggregation even without an Ordering 4b7a5ce [Matei Zaharia] More tests, and ability to sort data if a total ordering is given e1f84be [Matei Zaharia] Fix disk block manager test 5a40a1c [Matei Zaharia] More tests 614f1b4 [Matei Zaharia] Add spill metrics to map tasks cc52caf [Matei Zaharia] Add more error handling and tests for error cases bbf359d [Matei Zaharia] More work 3a56341 [Matei Zaharia] More partial work towards sort-based shuffle 7a0895d [Matei Zaharia] Some more partial work towards sort-based shuffle b615476 [Matei Zaharia] Scaffolding for sort-based shuffle
* [SPARK-2734][SQL] Remove tables from cache when DROP TABLE is run.Michael Armbrust2014-07-304-1/+74
| | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #1650 from marmbrus/dropCached and squashes the following commits: e6ab80b [Michael Armbrust] Support if exists. 83426c6 [Michael Armbrust] Remove tables from cache when DROP TABLE is run.
* SPARK-2749 [BUILD]. Spark SQL Java tests aren't compiling in Jenkins' Maven ↵Sean Owen2014-07-301-0/+5
| | | | | | | | | | | | | | | | | | | | | | | | | | | builds; missing junit:junit dep The Maven-based builds in the build matrix have been failing for a few days: https://amplab.cs.berkeley.edu/jenkins/view/Spark/ On inspection, it looks like the Spark SQL Java tests don't compile: https://amplab.cs.berkeley.edu/jenkins/view/Spark/job/Spark-Master-Maven-pre-YARN/hadoop.version=1.0.4,label=centos/244/consoleFull I confirmed it by repeating the command vs master: `mvn -Dhadoop.version=1.0.4 -Dlabel=centos -DskipTests clean package` The problem is that this module doesn't depend on JUnit. In fact, none of the modules do, but `com.novocode:junit-interface` (the SBT-JUnit bridge) pulls it in, in most places. However this module doesn't depend on `com.novocode:junit-interface` Adding the `junit:junit` dependency fixes the compile problem. In fact, the other modules with Java tests should probably depend on it explicitly instead of happening to get it via `com.novocode:junit-interface`, since that is a bit SBT/Scala-specific (and I am not even sure it's needed). Author: Sean Owen <srowen@gmail.com> Closes #1660 from srowen/SPARK-2749 and squashes the following commits: 858ff7c [Sean Owen] Add explicit junit dep to other modules with Java tests for robustness 9636794 [Sean Owen] Add junit dep so that Spark SQL Java tests compile
* [SPARK-2179][SQL] Public API for DataTypes and SchemaYin Huai2014-07-3058-370/+2886
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | The current PR contains the following changes: * Expose `DataType`s in the sql package (internal details are private to sql). * Users can create Rows. * Introduce `applySchema` to create a `SchemaRDD` by applying a `schema: StructType` to an `RDD[Row]`. * Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`. * `ScalaReflection.typeOfObject` provides a way to infer the Catalyst data type based on an object. Also, we can compose `typeOfObject` with some custom logics to form a new function to infer the data type (for different use cases). * `JsonRDD` has been refactored to use changes introduced by this PR. * Add a field `containsNull` to `ArrayType`. So, we can explicitly mark if an `ArrayType` can contain null values. The default value of `containsNull` is `false`. New APIs are introduced in the sql package object and SQLContext. You can find the scaladoc at [sql package object](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.package) and [SQLContext](http://yhuai.github.io/site/api/scala/index.html#org.apache.spark.sql.SQLContext). An example of using `applySchema` is shown below. ```scala import org.apache.spark.sql._ val sqlContext = new org.apache.spark.sql.SQLContext(sc) val schema = StructType( StructField("name", StringType, false) :: StructField("age", IntegerType, true) :: Nil) val people = sc.textFile("examples/src/main/resources/people.txt").map(_.split(",")).map(p => Row(p(0), p(1).trim.toInt)) val peopleSchemaRDD = sqlContext. applySchema(people, schema) peopleSchemaRDD.printSchema // root // |-- name: string (nullable = false) // |-- age: integer (nullable = true) peopleSchemaRDD.registerAsTable("people") sqlContext.sql("select name from people").collect.foreach(println) ``` I will add new contents to the SQL programming guide later. JIRA: https://issues.apache.org/jira/browse/SPARK-2179 Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1346 from yhuai/dataTypeAndSchema and squashes the following commits: 1d45977 [Yin Huai] Clean up. a6e08b4 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema c712fbf [Yin Huai] Converts types of values based on defined schema. 4ceeb66 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema e5f8df5 [Yin Huai] Scaladoc. 122d1e7 [Yin Huai] Address comments. 03bfd95 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 2476ed0 [Yin Huai] Minor updates. ab71f21 [Yin Huai] Format. fc2bed1 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema bd40a33 [Yin Huai] Address comments. 991f860 [Yin Huai] Move "asJavaDataType" and "asScalaDataType" to DataTypeConversions.scala. 1cb35fe [Yin Huai] Add "valueContainsNull" to MapType. 3edb3ae [Yin Huai] Python doc. 692c0b9 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 1d93395 [Yin Huai] Python APIs. 246da96 [Yin Huai] Add java data type APIs to javadoc index. 1db9531 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema d48fc7b [Yin Huai] Minor updates. 33c4fec [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema b9f3071 [Yin Huai] Java API for applySchema. 1c9f33c [Yin Huai] Java APIs for DataTypes and Row. 624765c [Yin Huai] Tests for applySchema. aa92e84 [Yin Huai] Update data type tests. 8da1a17 [Yin Huai] Add Row.fromSeq. 9c99bc0 [Yin Huai] Several minor updates. 1d9c13a [Yin Huai] Update applySchema API. 85e9b51 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema e495e4e [Yin Huai] More comments. 42d47a3 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema c3f4a02 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 2e58dbd [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema b8b7db4 [Yin Huai] 1. Move sql package object and package-info to sql-core. 2. Minor updates on APIs. 3. Update scala doc. 68525a2 [Yin Huai] Update JSON unit test. 3209108 [Yin Huai] Add unit tests. dcaf22f [Yin Huai] Add a field containsNull to ArrayType to indicate if an array can contain null values or not. If an ArrayType is constructed by "ArrayType(elementType)" (the existing constructor), the value of containsNull is false. 9168b83 [Yin Huai] Update comments. fc649d7 [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema eca7d04 [Yin Huai] Add two apply methods which will be used to extract StructField(s) from a StructType. 949d6bb [Yin Huai] When creating a SchemaRDD for a JSON dataset, users can apply an existing schema. 7a6a7e5 [Yin Huai] Fix bug introduced by the change made on SQLContext.inferSchema. 43a45e1 [Yin Huai] Remove sql.util.package introduced in a previous commit. 0266761 [Yin Huai] Format 03eec4c [Yin Huai] Merge remote-tracking branch 'upstream/master' into dataTypeAndSchema 90460ac [Yin Huai] Infer the Catalyst data type from an object and cast a data value to the expected type. 3fa0df5 [Yin Huai] Provide easier ways to construct a StructType. 16be3e5 [Yin Huai] This commit contains three changes: * Expose `DataType`s in the sql package (internal details are private to sql). * Introduce `createSchemaRDD` to create a `SchemaRDD` from an `RDD` with a provided schema (represented by a `StructType`) and a provided function to construct `Row`, * Add a function `simpleString` to every `DataType`. Also, the schema represented by a `StructType` can be visualized by `printSchema`.
* [SQL] Handle null values in debug()Michael Armbrust2014-07-291-1/+3
| | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #1646 from marmbrus/nullDebug and squashes the following commits: 49050a8 [Michael Armbrust] Handle null values in debug()
* [SPARK-2054][SQL] Code Generation for Expression EvaluationMichael Armbrust2014-07-2951-294/+1871
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | Adds a new method for evaluating expressions using code that is generated though Scala reflection. This functionality is configured by the SQLConf option `spark.sql.codegen` and is currently turned off by default. Evaluation can be done in several specialized ways: - *Projection* - Given an input row, produce a new row from a set of expressions that define each column in terms of the input row. This can either produce a new Row object or perform the projection in-place on an existing Row (MutableProjection). - *Ordering* - Compares two rows based on a list of `SortOrder` expressions - *Condition* - Returns `true` or `false` given an input row. For each of the above operations there is both a Generated and Interpreted version. When generation for a given expression type is undefined, the code generator falls back on calling the `eval` function of the expression class. Even without custom code, there is still a potential speed up, as loops are unrolled and code can still be inlined by JIT. This PR also contains a new type of Aggregation operator, `GeneratedAggregate`, that performs aggregation by using generated `Projection` code. Currently the required expression rewriting only works for simple aggregations like `SUM` and `COUNT`. This functionality will be extended in a future PR. This PR also performs several clean ups that simplified the implementation: - The notion of `Binding` all expressions in a tree automatically before query execution has been removed. Instead it is the responsibly of an operator to provide the input schema when creating one of the specialized evaluators defined above. In cases when the standard eval method is going to be called, binding can still be done manually using `BindReferences`. There are a few reasons for this change: First, there were many operators where it just didn't work before. For example, operators with more than one child, and operators like aggregation that do significant rewriting of the expression. Second, the semantics of equality with `BoundReferences` are broken. Specifically, we have had a few bugs where partitioning breaks because of the binding. - A copy of the current `SQLContext` is automatically propagated to all `SparkPlan` nodes by the query planner. Before this was done ad-hoc for the nodes that needed this. However, this required a lot of boilerplate as one had to always remember to make it `transient` and also had to modify the `otherCopyArgs`. Author: Michael Armbrust <michael@databricks.com> Closes #993 from marmbrus/newCodeGen and squashes the following commits: 96ef82c [Michael Armbrust] Merge remote-tracking branch 'apache/master' into newCodeGen f34122d [Michael Armbrust] Merge remote-tracking branch 'apache/master' into newCodeGen 67b1c48 [Michael Armbrust] Use conf variable in SQLConf object 4bdc42c [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen 41a40c9 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen de22aac [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen fed3634 [Michael Armbrust] Inspectors are not serializable. ef8d42b [Michael Armbrust] comments 533fdfd [Michael Armbrust] More logging of expression rewriting for GeneratedAggregate. 3cd773e [Michael Armbrust] Allow codegen for Generate. 64b2ee1 [Michael Armbrust] Implement copy 3587460 [Michael Armbrust] Drop unused string builder function. 9cce346 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen 1a61293 [Michael Armbrust] Address review comments. 0672e8a [Michael Armbrust] Address comments. 1ec2d6e [Michael Armbrust] Address comments 033abc6 [Michael Armbrust] off by default 4771fab [Michael Armbrust] Docs, more test coverage. d30fee2 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen d2ad5c5 [Michael Armbrust] Refactor putting SQLContext into SparkPlan. Fix ordering, other test cases. be2cd6b [Michael Armbrust] WIP: Remove old method for reference binding, more work on configuration. bc88ecd [Michael Armbrust] Style 6cc97ca [Michael Armbrust] Merge remote-tracking branch 'origin/master' into newCodeGen 4220f1e [Michael Armbrust] Better config, docs, etc. ca6cc6b [Michael Armbrust] WIP 9d67d85 [Michael Armbrust] Fix hive planner fc522d5 [Michael Armbrust] Hook generated aggregation in to the planner. e742640 [Michael Armbrust] Remove unneeded changes and code. 675e679 [Michael Armbrust] Upgrade paradise. 0093376 [Michael Armbrust] Comment / indenting cleanup. d81f998 [Michael Armbrust] include schema for binding. 0e889e8 [Michael Armbrust] Use typeOf instead tq f623ffd [Michael Armbrust] Quiet logging from test suite. efad14f [Michael Armbrust] Remove some half finished functions. 92e74a4 [Michael Armbrust] add overrides a2b5408 [Michael Armbrust] WIP: Code generation with scala reflection.
* [SPARK-2631][SQL] Use SQLConf to configure in-memory columnar cachingMichael Armbrust2014-07-292-2/+4
| | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #1638 from marmbrus/cachedConfig and squashes the following commits: 2362082 [Michael Armbrust] Use SQLConf to configure in-memory columnar caching
* [SPARK-2716][SQL] Don't check resolved for having filters.Michael Armbrust2014-07-293-1/+140
| | | | | | | | | | For queries like `... HAVING COUNT(*) > 9` the expression is always resolved since it contains no attributes. This was causing us to avoid doing the Having clause aggregation rewrite. Author: Michael Armbrust <michael@databricks.com> Closes #1640 from marmbrus/havingNoRef and squashes the following commits: 92d3901 [Michael Armbrust] Don't check resolved for having filters.
* [SPARK-2393][SQL] Cost estimation optimization framework for Catalyst ↵Zongheng Yang2014-07-2918-124/+256
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | logical plans & sample usage. The idea is that every Catalyst logical plan gets hold of a Statistics class, the usage of which provides useful estimations on various statistics. See the implementations of `MetastoreRelation`. This patch also includes several usages of the estimation interface in the planner. For instance, we now use physical table sizes from the estimate interface to convert an equi-join to a broadcast join (when doing so is beneficial, as determined by a size threshold). Finally, there are a couple minor accompanying changes including: - Remove the not-in-use `BaseRelation`. - Make SparkLogicalPlan take a `SQLContext` in the second param list. Author: Zongheng Yang <zongheng.y@gmail.com> Closes #1238 from concretevitamin/estimates and squashes the following commits: 329071d [Zongheng Yang] Address review comments; turn config name from string to field in SQLConf. 8663e84 [Zongheng Yang] Use BigInt for stat; for logical leaves, by default throw an exception. 2f2fb89 [Zongheng Yang] Fix statistics for SparkLogicalPlan. 9951305 [Zongheng Yang] Remove childrenStats. 16fc60a [Zongheng Yang] Avoid calling statistics on plans if auto join conversion is disabled. 8bd2816 [Zongheng Yang] Add a note on performance of statistics. 6e594b8 [Zongheng Yang] Get size info from metastore for MetastoreRelation. 01b7a3e [Zongheng Yang] Update scaladoc for a field and move it to @param section. 549061c [Zongheng Yang] Remove numTuples in Statistics for now. 729a8e2 [Zongheng Yang] Update docs to be more explicit. 573e644 [Zongheng Yang] Remove singleton SQLConf and move back `settings` to the trait. 2d99eb5 [Zongheng Yang] {Cleanup, use synchronized in, enrich} StatisticsSuite. ca5b825 [Zongheng Yang] Inject SQLContext into SparkLogicalPlan, removing SQLConf mixin from it. 43d38a6 [Zongheng Yang] Revert optimization for BroadcastNestedLoopJoin (this fixes tests). 0ef9e5b [Zongheng Yang] Use multiplication instead of sum for default estimates. 4ef0d26 [Zongheng Yang] Make Statistics a case class. 3ba8f3e [Zongheng Yang] Add comment. e5bcf5b [Zongheng Yang] Fix optimization conditions & update scala docs to explain. 7d9216a [Zongheng Yang] Apply estimation to planning ShuffleHashJoin & BroadcastNestedLoopJoin. 73cde01 [Zongheng Yang] Move SQLConf back. Assign default sizeInBytes to SparkLogicalPlan. 73412be [Zongheng Yang] Move SQLConf to Catalyst & add default val for sizeInBytes. 7a60ab7 [Zongheng Yang] s/Estimates/Statistics, s/cardinality/numTuples. de3ae13 [Zongheng Yang] Add parquetAfter() properly in test. dcff9bd [Zongheng Yang] Cleanups. 84301a4 [Zongheng Yang] Refactors. 5bf5586 [Zongheng Yang] Typo. 56a8e6e [Zongheng Yang] Prototype impl of estimations for Catalyst logical plans.
* [SPARK-2674] [SQL] [PySpark] support datetime type for SchemaRDDDavies Liu2014-07-292-32/+54
| | | | | | | | | | | | | | | Datetime and time in Python will be converted into java.util.Calendar after serialization, it will be converted into java.sql.Timestamp during inferSchema(). In javaToPython(), Timestamp will be converted into Calendar, then be converted into datetime in Python after pickling. Author: Davies Liu <davies.liu@gmail.com> Closes #1601 from davies/date and squashes the following commits: f0599b0 [Davies Liu] remove tests for sets and tuple in sql, fix list of list c9d607a [Davies Liu] convert datetype for runtime 709d40d [Davies Liu] remove brackets 96db384 [Davies Liu] support datetime type for SchemaRDD
* [SPARK-2730][SQL] When retrieving a value from a Map, GetItem evaluates key ↵Yin Huai2014-07-291-1/+0
| | | | | | | | | | | | twice JIRA: https://issues.apache.org/jira/browse/SPARK-2730 Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1637 from yhuai/SPARK-2730 and squashes the following commits: 1a9f24e [Yin Huai] Remove unnecessary key evaluation.
* [SQL]change some test listsDaoyuan2014-07-291-5/+1
| | | | | | | | | | | | | 1. there's no `hook_context.q` but a `hook_context_cs.q` in query folder 2. there's no `compute_stats_table.q` in query folder 3. there's no `having1.q` in query folder 4. `udf_E` and `udf_PI` appear twice in white list Author: Daoyuan <daoyuan.wang@intel.com> Closes #1634 from adrian-wang/testcases and squashes the following commits: d7482ce [Daoyuan] change some test lists
* Minor indentation and comment typo fixes.Aaron Staple2014-07-292-13/+13
| | | | | | | | | Author: Aaron Staple <astaple@gmail.com> Closes #1630 from staple/minor and squashes the following commits: 6f295a2 [Aaron Staple] Fix typos in comment about ExprId. 8566467 [Aaron Staple] Fix off by one column indentation in SqlParser.
* Excess judgmentYadong Qi2014-07-281-1/+1
| | | | | | | | Author: Yadong Qi <qiyadong2010@gmail.com> Closes #1629 from watermen/bug-fix2 and squashes the following commits: 59b7237 [Yadong Qi] Update HiveQl.scala
* [SPARK-2410][SQL] Merging Hive Thrift/JDBC server (with Maven profile fix)Cheng Lian2014-07-2823-54/+1403
| | | | | | | | | | | | | | | JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) Another try for #1399 & #1600. Those two PR breaks Jenkins builds because we made a separate profile `hive-thriftserver` in sub-project `assembly`, but the `hive-thriftserver` module is defined outside the `hive-thriftserver` profile. Thus every time a pull request that doesn't touch SQL code will also execute test suites defined in `hive-thriftserver`, but tests fail because related .class files are not included in the assembly jar. In the most recent commit, module `hive-thriftserver` is moved into its own profile to fix this problem. All previous commits are squashed for clarity. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1620 from liancheng/jdbc-with-maven-fix and squashes the following commits: 629988e [Cheng Lian] Moved hive-thriftserver module definition into its own profile ec3c7a7 [Cheng Lian] Cherry picked the Hive Thrift server
* [SPARK-2523] [SQL] Hadoop table scan bug fixingCheng Hao2014-07-284-115/+138
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | In HiveTableScan.scala, ObjectInspector was created for all of the partition based records, which probably causes ClassCastException if the object inspector is not identical among table & partitions. This is the follow up with: https://github.com/apache/spark/pull/1408 https://github.com/apache/spark/pull/1390 I've run a micro benchmark in my local with 15000000 records totally, and got the result as below: With This Patch | Partition-Based Table | Non-Partition-Based Table ------------ | ------------- | ------------- No | 1927 ms | 1885 ms Yes | 1541 ms | 1524 ms It showed this patch will also improve the performance. PS: the benchmark code is also attached. (thanks liancheng ) ``` package org.apache.spark.sql.hive import org.apache.spark.SparkContext import org.apache.spark.SparkConf import org.apache.spark.sql._ object HiveTableScanPrepare extends App { case class Record(key: String, value: String) val sparkContext = new SparkContext( new SparkConf() .setMaster("local") .setAppName(getClass.getSimpleName.stripSuffix("$"))) val hiveContext = new LocalHiveContext(sparkContext) val rdd = sparkContext.parallelize((1 to 3000000).map(i => Record(s"$i", s"val_$i"))) import hiveContext._ hql("SHOW TABLES") hql("DROP TABLE if exists part_scan_test") hql("DROP TABLE if exists scan_test") hql("DROP TABLE if exists records") rdd.registerAsTable("records") hql("""CREATE TABLE part_scan_test (key STRING, value STRING) PARTITIONED BY (part1 string, part2 STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) hql("""CREATE TABLE scan_test (key STRING, value STRING) | ROW FORMAT SERDE | 'org.apache.hadoop.hive.serde2.columnar.LazyBinaryColumnarSerDe' | STORED AS RCFILE """.stripMargin) for (part1 <- 2000 until 2001) { for (part2 <- 1 to 5) { hql(s"""from records | insert into table part_scan_test PARTITION (part1='$part1', part2='2010-01-$part2') | select key, value """.stripMargin) hql(s"""from records | insert into table scan_test select key, value """.stripMargin) } } } object HiveTableScanTest extends App { val sparkContext = new SparkContext( new SparkConf() .setMaster("local") .setAppName(getClass.getSimpleName.stripSuffix("$"))) val hiveContext = new LocalHiveContext(sparkContext) import hiveContext._ hql("SHOW TABLES") val part_scan_test = hql("select key, value from part_scan_test") val scan_test = hql("select key, value from scan_test") val r_part_scan_test = (0 to 5).map(i => benchmark(part_scan_test)) val r_scan_test = (0 to 5).map(i => benchmark(scan_test)) println("Scanning Partition-Based Table") r_part_scan_test.foreach(printResult) println("Scanning Non-Partition-Based Table") r_scan_test.foreach(printResult) def printResult(result: (Long, Long)) { println(s"Duration: ${result._1} ms Result: ${result._2}") } def benchmark(srdd: SchemaRDD) = { val begin = System.currentTimeMillis() val result = srdd.count() val end = System.currentTimeMillis() ((end - begin), result) } } ``` Author: Cheng Hao <hao.cheng@intel.com> Closes #1439 from chenghao-intel/hadoop_table_scan and squashes the following commits: 888968f [Cheng Hao] Fix issues in code style 27540ba [Cheng Hao] Fix the TableScan Bug while partition serde differs 40a24a7 [Cheng Hao] Add Unit Test
* Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"Patrick Wendell2014-07-2723-1403/+54
| | | | This reverts commit f6ff2a61d00d12481bfb211ae13d6992daacdcc2.
* [SPARK-2410][SQL] Merging Hive Thrift/JDBC serverCheng Lian2014-07-2723-54/+1403
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | (This is a replacement of #1399, trying to fix potential `HiveThriftServer2` port collision between parallel builds. Please refer to [these comments](https://github.com/apache/spark/pull/1399#issuecomment-50212572) for details.) JIRA issue: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) Merging the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc). Thanks chenghao-intel for his initial contribution of the Spark SQL CLI. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1600 from liancheng/jdbc and squashes the following commits: ac4618b [Cheng Lian] Uses random port for HiveThriftServer2 to avoid collision with parallel builds 090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR 21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd] 199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver 1083e9d [Cheng Lian] Fixed failed test suites 7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic 9cc0f06 [Cheng Lian] Starts beeline with spark-submit cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile 061880f [Cheng Lian] Addressed all comments by @pwendell 7755062 [Cheng Lian] Adapts test suites to spark-submit settings 40bafef [Cheng Lian] Fixed more license header issues e214aab [Cheng Lian] Added missing license headers b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft 3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit 61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit 2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server
* [SPARK-2659][SQL] Fix division semantics for hiveMichael Armbrust2014-07-257-9/+27
| | | | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #1557 from marmbrus/fixDivision and squashes the following commits: b85077f [Michael Armbrust] Fix unit tests. af98f29 [Michael Armbrust] Change DIV to long type 0c29ae8 [Michael Armbrust] Fix division semantics for hive
* Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"Michael Armbrust2014-07-2523-1395/+54
| | | | | | | | | | | | This reverts commit 06dc0d2c6b69c5d59b4d194ced2ac85bfe2e05e2. #1399 is making Jenkins fail. We should investigate and put this back after its passing tests. Author: Michael Armbrust <michael@databricks.com> Closes #1594 from marmbrus/revertJDBC and squashes the following commits: 59748da [Michael Armbrust] Revert "[SPARK-2410][SQL] Merging Hive Thrift/JDBC server"
* [SQL]Update HiveMetastoreCatalog.scalabaishuo(白硕)2014-07-251-1/+1
| | | | | | | | | | | I think it's better to defined hiveQlTable as a val Author: baishuo(白硕) <vc_java@hotmail.com> Closes #1569 from baishuo/patch-1 and squashes the following commits: dc2f895 [baishuo(白硕)] Update HiveMetastoreCatalog.scala a7b32a2 [baishuo(白硕)] Update HiveMetastoreCatalog.scala
* [SPARK-2410][SQL] Merging Hive Thrift/JDBC serverCheng Lian2014-07-2523-54/+1395
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | JIRA issue: - Main: [SPARK-2410](https://issues.apache.org/jira/browse/SPARK-2410) - Related: [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678) Cherry picked the Hive Thrift/JDBC server from [branch-1.0-jdbc](https://github.com/apache/spark/tree/branch-1.0-jdbc). (Thanks chenghao-intel for his initial contribution of the Spark SQL CLI.) TODO - [x] Use `spark-submit` to launch the server, the CLI and beeline - [x] Migration guideline draft for Shark users ---- Hit by a bug in `SparkSubmitArguments` while working on this PR: all application options that are recognized by `SparkSubmitArguments` are stolen as `SparkSubmit` options. For example: ```bash $ spark-submit --class org.apache.hive.beeline.BeeLine spark-internal --help ``` This actually shows usage information of `SparkSubmit` rather than `BeeLine`. ~~Fixed this bug here since the `spark-internal` related stuff also touches `SparkSubmitArguments` and I'd like to avoid conflict.~~ **UPDATE** The bug mentioned above is now tracked by [SPARK-2678](https://issues.apache.org/jira/browse/SPARK-2678). Decided to revert changes to this bug since it involves more subtle considerations and worth a separate PR. Author: Cheng Lian <lian.cs.zju@gmail.com> Closes #1399 from liancheng/thriftserver and squashes the following commits: 090beea [Cheng Lian] Revert changes related to SPARK-2678, decided to move them to another PR 21c6cf4 [Cheng Lian] Updated Spark SQL programming guide docs fe0af31 [Cheng Lian] Reordered spark-submit options in spark-shell[.cmd] 199e3fb [Cheng Lian] Disabled MIMA for hive-thriftserver 1083e9d [Cheng Lian] Fixed failed test suites 7db82a1 [Cheng Lian] Fixed spark-submit application options handling logic 9cc0f06 [Cheng Lian] Starts beeline with spark-submit cfcf461 [Cheng Lian] Updated documents and build scripts for the newly added hive-thriftserver profile 061880f [Cheng Lian] Addressed all comments by @pwendell 7755062 [Cheng Lian] Adapts test suites to spark-submit settings 40bafef [Cheng Lian] Fixed more license header issues e214aab [Cheng Lian] Added missing license headers b8905ba [Cheng Lian] Fixed minor issues in spark-sql and start-thriftserver.sh f975d22 [Cheng Lian] Updated docs for Hive compatibility and Shark migration guide draft 3ad4e75 [Cheng Lian] Starts spark-sql shell with spark-submit a5310d1 [Cheng Lian] Make HiveThriftServer2 play well with spark-submit 61f39f4 [Cheng Lian] Starts Hive Thrift server via spark-submit 2c4c539 [Cheng Lian] Cherry picked the Hive Thrift server
* [SPARK-2665] [SQL] Add EqualNS & Unit TestsCheng Hao2014-07-2558-3/+683
| | | | | | | | | | | | | Hive Supports the operator "<=>", which returns same result with EQUAL(=) operator for non-null operands, but returns TRUE if both are NULL, FALSE if one of the them is NULL. Author: Cheng Hao <hao.cheng@intel.com> Closes #1570 from chenghao-intel/equalns and squashes the following commits: 8d6c789 [Cheng Hao] Remove the test case orc_predicate_pushdown 5b2ca88 [Cheng Hao] Add cases into whitelist 8e66cdd [Cheng Hao] Rename the EqualNSTo ==> EqualNullSafe 7af4b0b [Cheng Hao] Add EqualNS & Unit Tests
* [SPARK-2603][SQL] Remove unnecessary toMap and toList in converting Java ↵Yin Huai2014-07-241-6/+6
| | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | | collections to Scala collections JsonRDD.scala In JsonRDD.scalafy, we are using toMap/toList to convert a Java Map/List to a Scala one. These two operations are pretty expensive because they read elements from a Java Map/List and then load to a Scala Map/List. We can use Scala wrappers to wrap those Java collections instead of using toMap/toList. I did a quick test to see the performance. I had a 2.9GB cached RDD[String] storing one JSON object per record (twitter dataset). My simple test program is attached below. ```scala val sqlContext = new org.apache.spark.sql.SQLContext(sc) import sqlContext._ val jsonData = sc.textFile("...") jsonData.cache.count val jsonSchemaRDD = sqlContext.jsonRDD(jsonData) jsonSchemaRDD.registerAsTable("jt") sqlContext.sql("select count(*) from jt").collect ``` Stages for the schema inference and the table scan both had 48 tasks. These tasks were executed sequentially. For the current implementation, scanning the JSON dataset will materialize values of all fields of a record. The inferred schema of the dataset can be accessed at https://gist.github.com/yhuai/05fe8a57c638c6666f8d. From the result, there was no significant difference on running `jsonRDD`. For the simple aggregation query, results are attached below. ``` Original: Run 1: 26.1s Run 2: 27.03s Run 3: 27.035s With this change: Run 1: 21.086s Run 2: 21.035s Run 3: 21.029s ``` JIRA: https://issues.apache.org/jira/browse/SPARK-2603 Author: Yin Huai <huai@cse.ohio-state.edu> Closes #1504 from yhuai/removeToMapToList and squashes the following commits: 6831b77 [Yin Huai] Fix failed tests. 09b9bca [Yin Huai] Merge remote-tracking branch 'upstream/master' into removeToMapToList d1abdb8 [Yin Huai] Remove unnecessary toMap and toList.
* [SPARK-2658][SQL] Add rule for true = 1.Michael Armbrust2014-07-233-1/+24
| | | | | | | | Author: Michael Armbrust <michael@databricks.com> Closes #1556 from marmbrus/fixBooleanEqualsOne and squashes the following commits: ad8edd4 [Michael Armbrust] Add rule for true = 1 and false = 0.
* [SPARK-2484][SQL] Build should not run hivecompatibility tests by default.witgo2014-07-232-0/+30
| | | | | | | | Author: witgo <witgo@qq.com> Closes #1403 from witgo/hive_compatibility and squashes the following commits: 4e5ecdb [witgo] The default does not run hive compatibility tests