spark git commit: [SPARK-7470] [SQL] Spark shell SQLContext crashes without hive
Repository: spark Updated Branches: refs/heads/master 3af423c92 - 714db2ef5 [SPARK-7470] [SQL] Spark shell SQLContext crashes without hive This only happens if you have `SPARK_PREPEND_CLASSES` set. Then I built it with `build/sbt clean assembly compile` and just ran it with `bin/spark-shell`. ``` ... 15/05/07 17:07:30 INFO EventLoggingListener: Logging events to file:/tmp/spark-events/local-1431043649919 15/05/07 17:07:30 INFO SparkILoop: Created spark context.. Spark context available as sc. java.lang.NoClassDefFoundError: org/apache/hadoop/hive/conf/HiveConf at java.lang.Class.getDeclaredConstructors0(Native Method) at java.lang.Class.privateGetDeclaredConstructors(Class.java:2493) at java.lang.Class.getConstructor0(Class.java:2803) ... Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.conf.HiveConf at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) ... 52 more console:10: error: not found: value sqlContext import sqlContext.implicits._ ^ console:10: error: not found: value sqlContext import sqlContext.sql ^ ``` yhuai marmbrus Author: Andrew Or and...@databricks.com Closes #5997 from andrewor14/sql-shell-crash and squashes the following commits: 61147e6 [Andrew Or] Also expect NoClassDefFoundError Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/714db2ef Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/714db2ef Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/714db2ef Branch: refs/heads/master Commit: 714db2ef52c0fe34418e252e5a6f220337022046 Parents: 3af423c Author: Andrew Or and...@databricks.com Authored: Thu May 7 22:32:13 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 22:32:13 2015 -0700 -- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/714db2ef/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala -- diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 8dc0e0c..488f3a9 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1028,7 +1028,7 @@ class SparkILoop( logInfo(Created sql context (with Hive support)..) } catch { - case cnf: java.lang.ClassNotFoundException = + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError = sqlContext = new SQLContext(sparkContext) logInfo(Created sql context..) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7232] [SQL] Add a Substitution batch for spark sql analyzer
Repository: spark Updated Branches: refs/heads/branch-1.4 1a3e9e982 - bb5872f2d [SPARK-7232] [SQL] Add a Substitution batch for spark sql analyzer Added a new batch named `Substitution` before `Resolution` batch. The motivation for this is there are kind of cases we want to do some substitution on the parsed logical plan before resolve it. Consider this two cases: 1 CTE, for cte we first build a row logical plan ``` 'With Map(q1 - 'Subquery q1 'Project ['key] 'UnresolvedRelation [src], None) 'Project [*] 'Filter ('key = 5) 'UnresolvedRelation [q1], None ``` In `With` logicalplan here is a map stored the (`q1- subquery`), we want first take off the with command and substitute the `q1` of `UnresolvedRelation` by the `subquery` 2 Another example is Window function, in window function user may define some windows, we also need substitute the window name of child by the concrete window. this should also done in the Substitution batch. Author: wangfei wangf...@huawei.com Closes #5776 from scwf/addbatch and squashes the following commits: d4b962f [wangfei] added WindowsSubstitution 70f6932 [wangfei] Merge branch 'master' of https://github.com/apache/spark into addbatch ecaeafb [wangfei] address yhuai's comments 553005a [wangfei] fix test case 0c54798 [wangfei] address comments 29f [wangfei] fix compile 1c9a092 [wangfei] added Substitution bastch (cherry picked from commit f496bf3c539a873ffdf3aa803847ef7b50135bd7) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bb5872f2 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bb5872f2 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bb5872f2 Branch: refs/heads/branch-1.4 Commit: bb5872f2d099bd6d679c20c2a4ff28eb35143e6d Parents: 1a3e9e9 Author: wangfei wangf...@huawei.com Authored: Thu May 7 22:55:42 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 22:57:15 2015 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 98 1 file changed, 60 insertions(+), 38 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bb5872f2/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 7e46ad8..bb7913e 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 @@ -55,6 +55,10 @@ class Analyzer( val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( +Batch(Substitution, fixedPoint, + CTESubstitution :: + WindowsSubstitution :: + Nil : _*), Batch(Resolution, fixedPoint, ResolveRelations :: ResolveReferences :: @@ -72,6 +76,55 @@ class Analyzer( ) /** + * Substitute child plan with cte definitions + */ + object CTESubstitution extends Rule[LogicalPlan] { +// TODO allow subquery to define CTE +def apply(plan: LogicalPlan): LogicalPlan = plan match { + case With(child, relations) = substituteCTE(child, relations) + case other = other +} + +def substituteCTE(plan: LogicalPlan, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { + plan transform { +// In hive, if there is same table name in database and CTE definition, +// hive will use the table in database, not the CTE one. +// Taking into account the reasonableness and the implementation complexity, +// here use the CTE definition first, check table name only and ignore database name +// see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info +case u : UnresolvedRelation = + val substituted = cteRelations.get(u.tableIdentifier.last).map { relation = +val withAlias = u.alias.map(Subquery(_, relation)) +withAlias.getOrElse(relation) + } + substituted.getOrElse(u) + } +} + } + + /** + * Substitute child plan with WindowSpecDefinitions. + */ + object WindowsSubstitution extends Rule[LogicalPlan] { +def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) = +child.transform { + case plan = plan.transformExpressions { +case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) = +
spark git commit: [SPARK-7232] [SQL] Add a Substitution batch for spark sql analyzer
Repository: spark Updated Branches: refs/heads/master 714db2ef5 - f496bf3c5 [SPARK-7232] [SQL] Add a Substitution batch for spark sql analyzer Added a new batch named `Substitution` before `Resolution` batch. The motivation for this is there are kind of cases we want to do some substitution on the parsed logical plan before resolve it. Consider this two cases: 1 CTE, for cte we first build a row logical plan ``` 'With Map(q1 - 'Subquery q1 'Project ['key] 'UnresolvedRelation [src], None) 'Project [*] 'Filter ('key = 5) 'UnresolvedRelation [q1], None ``` In `With` logicalplan here is a map stored the (`q1- subquery`), we want first take off the with command and substitute the `q1` of `UnresolvedRelation` by the `subquery` 2 Another example is Window function, in window function user may define some windows, we also need substitute the window name of child by the concrete window. this should also done in the Substitution batch. Author: wangfei wangf...@huawei.com Closes #5776 from scwf/addbatch and squashes the following commits: d4b962f [wangfei] added WindowsSubstitution 70f6932 [wangfei] Merge branch 'master' of https://github.com/apache/spark into addbatch ecaeafb [wangfei] address yhuai's comments 553005a [wangfei] fix test case 0c54798 [wangfei] address comments 29f [wangfei] fix compile 1c9a092 [wangfei] added Substitution bastch Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f496bf3c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f496bf3c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f496bf3c Branch: refs/heads/master Commit: f496bf3c539a873ffdf3aa803847ef7b50135bd7 Parents: 714db2e Author: wangfei wangf...@huawei.com Authored: Thu May 7 22:55:42 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 22:55:42 2015 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 98 1 file changed, 60 insertions(+), 38 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f496bf3c/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 7e46ad8..bb7913e 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 @@ -55,6 +55,10 @@ class Analyzer( val extendedResolutionRules: Seq[Rule[LogicalPlan]] = Nil lazy val batches: Seq[Batch] = Seq( +Batch(Substitution, fixedPoint, + CTESubstitution :: + WindowsSubstitution :: + Nil : _*), Batch(Resolution, fixedPoint, ResolveRelations :: ResolveReferences :: @@ -72,6 +76,55 @@ class Analyzer( ) /** + * Substitute child plan with cte definitions + */ + object CTESubstitution extends Rule[LogicalPlan] { +// TODO allow subquery to define CTE +def apply(plan: LogicalPlan): LogicalPlan = plan match { + case With(child, relations) = substituteCTE(child, relations) + case other = other +} + +def substituteCTE(plan: LogicalPlan, cteRelations: Map[String, LogicalPlan]): LogicalPlan = { + plan transform { +// In hive, if there is same table name in database and CTE definition, +// hive will use the table in database, not the CTE one. +// Taking into account the reasonableness and the implementation complexity, +// here use the CTE definition first, check table name only and ignore database name +// see https://github.com/apache/spark/pull/4929#discussion_r27186638 for more info +case u : UnresolvedRelation = + val substituted = cteRelations.get(u.tableIdentifier.last).map { relation = +val withAlias = u.alias.map(Subquery(_, relation)) +withAlias.getOrElse(relation) + } + substituted.getOrElse(u) + } +} + } + + /** + * Substitute child plan with WindowSpecDefinitions. + */ + object WindowsSubstitution extends Rule[LogicalPlan] { +def apply(plan: LogicalPlan): LogicalPlan = plan transform { + // Lookup WindowSpecDefinitions. This rule works with unresolved children. + case WithWindowDefinition(windowDefinitions, child) = +child.transform { + case plan = plan.transformExpressions { +case UnresolvedWindowExpression(c, WindowSpecReference(windowName)) = + val errorMessage = +sWindow specification $windowName is not defined in the WINDOW clause. +
[2/2] spark git commit: [SPARK-6908] [SQL] Use isolated Hive client
[SPARK-6908] [SQL] Use isolated Hive client This PR switches Spark SQL's Hive support to use the isolated hive client interface introduced by #5851, instead of directly interacting with the client. By using this isolated client we can now allow users to dynamically configure the version of Hive that they are connecting to by setting `spark.sql.hive.metastore.version` without the need recompile. This also greatly reduces the surface area for our interaction with the hive libraries, hopefully making it easier to support other versions in the future. Jars for the desired hive version can be configured using `spark.sql.hive.metastore.jars`, which accepts the following options: - a colon-separated list of jar files or directories for hive and hadoop. - `builtin` - attempt to discover the jars that were used to load Spark SQL and use those. This option is only valid when using the execution version of Hive. - `maven` - download the correct version of hive on demand from maven. By default, `builtin` is used for Hive 13. This PR also removes the test step for building against Hive 12, as this will no longer be required to talk to Hive 12 metastores. However, the full removal of the Shim is deferred until a later PR. Remaining TODOs: - Remove the Hive Shims and inline code for Hive 13. - Several HiveCompatibility tests are not yet passing. - `nullformatCTAS` - As detailed below, we now are handling CTAS parsing ourselves instead of hacking into the Hive semantic analyzer. However, we currently only handle the common cases and not things like CTAS where the null format is specified. - `combine1` now leaks state about compression somehow, breaking all subsequent tests. As such we currently add it to the blacklist - `part_inherit_tbl_props` and `part_inherit_tbl_props_with_star` do not work anymore. We are correctly propagating the information - load_dyn_part14.* - These tests pass when run on their own, but fail when run with all other tests. It seems our `RESET` mechanism may not be as robust as it used to be? Other required changes: - `CreateTableAsSelect` no longer carries parts of the HiveQL AST with it through the query execution pipeline. Instead, we parse CTAS during the HiveQL conversion and construct a `HiveTable`. The full parsing here is not yet complete as detailed above in the remaining TODOs. Since the operator is Hive specific, it is moved to the hive package. - `Command` is simplified to be a trait that simply acts as a marker for a LogicalPlan that should be eagerly evaluated. Author: Michael Armbrust mich...@databricks.com Closes #5876 from marmbrus/useIsolatedClient and squashes the following commits: 258d000 [Michael Armbrust] really really correct path handling e56fd4a [Michael Armbrust] getAbsolutePath 5a259f5 [Michael Armbrust] fix typos 81bb366 [Michael Armbrust] comments from vanzin 5f3945e [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient 4b5cd41 [Michael Armbrust] yin's comments f5de7de [Michael Armbrust] cleanup 11e9c72 [Michael Armbrust] better coverage in versions suite 7e8f010 [Michael Armbrust] better error messages and jar handling e7b3941 [Michael Armbrust] more permisive checking for function registration da91ba7 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient 5fe5894 [Michael Armbrust] fix serialization suite 81711c4 [Michael Armbrust] Initial support for running without maven 1d8ae44 [Michael Armbrust] fix final tests? 1c50813 [Michael Armbrust] more comments a3bee70 [Michael Armbrust] Merge remote-tracking branch 'origin/master' into useIsolatedClient a6f5df1 [Michael Armbrust] style ab07f7e [Michael Armbrust] WIP 4d8bf02 [Michael Armbrust] Remove hive 12 compilation 8843a25 [Michael Armbrust] [SPARK-6908] [SQL] Use isolated Hive client (cherry picked from commit cd1d4110cfffb413ab585cf1cc8f1264243cb393) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/05454fd8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/05454fd8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/05454fd8 Branch: refs/heads/branch-1.4 Commit: 05454fd8aef75b129cbbd0288f5089c5259f4a15 Parents: 2e8a141 Author: Michael Armbrust mich...@databricks.com Authored: Thu May 7 19:36:24 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 19:36:41 2015 -0700 -- dev/run-tests | 23 - project/MimaExcludes.scala | 2 + project/SparkBuild.scala| 9 +- .../catalyst/plans/logical/basicOperators.scala | 16 +- .../sql/catalyst/plans/logical/commands.scala | 8 +- .../spark/sql/catalyst/SqlParserSuite.scala | 6 +- .../scala/org/apache/spark/sql/DataFrame.scala | 1 -
[1/2] spark git commit: [SPARK-6908] [SQL] Use isolated Hive client
Repository: spark Updated Branches: refs/heads/branch-1.4 2e8a141b5 - 05454fd8a http://git-wip-us.apache.org/repos/asf/spark/blob/05454fd8/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index ea52fea..6bca9d0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.client import java.io.{BufferedReader, InputStreamReader, File, PrintStream} import java.net.URI -import java.util.{ArrayList = JArrayList} +import java.util.{ArrayList = JArrayList, Map = JMap, List = JList, Set = JSet} import scala.collection.JavaConversions._ import scala.language.reflectiveCalls @@ -27,6 +27,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.Database import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata @@ -54,19 +55,13 @@ import org.apache.spark.sql.execution.QueryExecutionException * @param config a collection of configuration options that will be added to the hive conf before *opening the hive client. */ -class ClientWrapper( +private[hive] class ClientWrapper( version: HiveVersion, config: Map[String, String]) extends ClientInterface with Logging with ReflectionMagic { - private val conf = new HiveConf(classOf[SessionState]) - config.foreach { case (k, v) = -logDebug(sHive Config: $k=$v) -conf.set(k, v) - } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. private val outputBuffer = new java.io.OutputStream { var pos: Int = 0 @@ -99,17 +94,31 @@ class ClientWrapper( val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) val ret = try { - val newState = new SessionState(conf) - SessionState.start(newState) - newState.out = new PrintStream(outputBuffer, true, UTF-8) - newState.err = new PrintStream(outputBuffer, true, UTF-8) - newState + val oldState = SessionState.get() + if (oldState == null) { +val initialConf = new HiveConf(classOf[SessionState]) +config.foreach { case (k, v) = + logDebug(sHive Config: $k=$v) + initialConf.set(k, v) +} +val newState = new SessionState(initialConf) +SessionState.start(newState) +newState.out = new PrintStream(outputBuffer, true, UTF-8) +newState.err = new PrintStream(outputBuffer, true, UTF-8) +newState + } else { +oldState + } } finally { Thread.currentThread().setContextClassLoader(original) } ret } + /** Returns the configuration for the current session. */ + def conf: HiveConf = SessionState.get().getConf + + // TODO: should be a def?s private val client = Hive.get(conf) /** @@ -133,6 +142,18 @@ class ClientWrapper( ret } + def setOut(stream: PrintStream): Unit = withHiveState { +state.out = stream + } + + def setInfo(stream: PrintStream): Unit = withHiveState { +state.info = stream + } + + def setError(stream: PrintStream): Unit = withHiveState { +state.err = stream + } + override def currentDatabase: String = withHiveState { state.getCurrentDatabase } @@ -171,14 +192,20 @@ class ClientWrapper( partitionColumns = h.getPartCols.map(f = HiveColumn(f.getName, f.getType, f.getComment)), properties = h.getParameters.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, -tableType = ManagedTable, // TODO +tableType = h.getTableType match { + case TableType.MANAGED_TABLE = ManagedTable + case TableType.EXTERNAL_TABLE = ExternalTable + case TableType.VIRTUAL_VIEW = VirtualView + case TableType.INDEX_TABLE = IndexTable +}, location = version match { case hive.v12 = Option(h.call[URI](getDataLocation)).map(_.toString) case hive.v13 = Option(h.call[Path](getDataLocation)).map(_.toString) }, inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), -serde = Option(h.getSerializationLib)).withClient(this) +serde = Option(h.getSerializationLib), +viewText = Option(h.getViewExpandedText)).withClient(this) } converted
[1/2] spark git commit: [SPARK-6908] [SQL] Use isolated Hive client
Repository: spark Updated Branches: refs/heads/master 22ab70e06 - cd1d4110c http://git-wip-us.apache.org/repos/asf/spark/blob/cd1d4110/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index ea52fea..6bca9d0 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -19,7 +19,7 @@ package org.apache.spark.sql.hive.client import java.io.{BufferedReader, InputStreamReader, File, PrintStream} import java.net.URI -import java.util.{ArrayList = JArrayList} +import java.util.{ArrayList = JArrayList, Map = JMap, List = JList, Set = JSet} import scala.collection.JavaConversions._ import scala.language.reflectiveCalls @@ -27,6 +27,7 @@ import scala.language.reflectiveCalls import org.apache.hadoop.fs.Path import org.apache.hadoop.hive.metastore.api.Database import org.apache.hadoop.hive.conf.HiveConf +import org.apache.hadoop.hive.metastore.TableType import org.apache.hadoop.hive.metastore.api import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hadoop.hive.ql.metadata @@ -54,19 +55,13 @@ import org.apache.spark.sql.execution.QueryExecutionException * @param config a collection of configuration options that will be added to the hive conf before *opening the hive client. */ -class ClientWrapper( +private[hive] class ClientWrapper( version: HiveVersion, config: Map[String, String]) extends ClientInterface with Logging with ReflectionMagic { - private val conf = new HiveConf(classOf[SessionState]) - config.foreach { case (k, v) = -logDebug(sHive Config: $k=$v) -conf.set(k, v) - } - // Circular buffer to hold what hive prints to STDOUT and ERR. Only printed when failures occur. private val outputBuffer = new java.io.OutputStream { var pos: Int = 0 @@ -99,17 +94,31 @@ class ClientWrapper( val original = Thread.currentThread().getContextClassLoader Thread.currentThread().setContextClassLoader(getClass.getClassLoader) val ret = try { - val newState = new SessionState(conf) - SessionState.start(newState) - newState.out = new PrintStream(outputBuffer, true, UTF-8) - newState.err = new PrintStream(outputBuffer, true, UTF-8) - newState + val oldState = SessionState.get() + if (oldState == null) { +val initialConf = new HiveConf(classOf[SessionState]) +config.foreach { case (k, v) = + logDebug(sHive Config: $k=$v) + initialConf.set(k, v) +} +val newState = new SessionState(initialConf) +SessionState.start(newState) +newState.out = new PrintStream(outputBuffer, true, UTF-8) +newState.err = new PrintStream(outputBuffer, true, UTF-8) +newState + } else { +oldState + } } finally { Thread.currentThread().setContextClassLoader(original) } ret } + /** Returns the configuration for the current session. */ + def conf: HiveConf = SessionState.get().getConf + + // TODO: should be a def?s private val client = Hive.get(conf) /** @@ -133,6 +142,18 @@ class ClientWrapper( ret } + def setOut(stream: PrintStream): Unit = withHiveState { +state.out = stream + } + + def setInfo(stream: PrintStream): Unit = withHiveState { +state.info = stream + } + + def setError(stream: PrintStream): Unit = withHiveState { +state.err = stream + } + override def currentDatabase: String = withHiveState { state.getCurrentDatabase } @@ -171,14 +192,20 @@ class ClientWrapper( partitionColumns = h.getPartCols.map(f = HiveColumn(f.getName, f.getType, f.getComment)), properties = h.getParameters.toMap, serdeProperties = h.getTTable.getSd.getSerdeInfo.getParameters.toMap, -tableType = ManagedTable, // TODO +tableType = h.getTableType match { + case TableType.MANAGED_TABLE = ManagedTable + case TableType.EXTERNAL_TABLE = ExternalTable + case TableType.VIRTUAL_VIEW = VirtualView + case TableType.INDEX_TABLE = IndexTable +}, location = version match { case hive.v12 = Option(h.call[URI](getDataLocation)).map(_.toString) case hive.v13 = Option(h.call[Path](getDataLocation)).map(_.toString) }, inputFormat = Option(h.getInputFormatClass).map(_.getName), outputFormat = Option(h.getOutputFormatClass).map(_.getName), -serde = Option(h.getSerializationLib)).withClient(this) +serde = Option(h.getSerializationLib), +viewText = Option(h.getViewExpandedText)).withClient(this) } converted }
spark git commit: [SPARK-6986] [SQL] Use Serializer2 in more cases.
Repository: spark Updated Branches: refs/heads/branch-1.4 28d423870 - 9d0d28940 [SPARK-6986] [SQL] Use Serializer2 in more cases. With https://github.com/apache/spark/commit/0a2b15ce43cf6096e1a7ae060b7c8a4010ce3b92, the serialization stream and deserialization stream has enough information to determine it is handling a key-value pari, a key, or a value. It is safe to use `SparkSqlSerializer2` in more cases. Author: Yin Huai yh...@databricks.com Closes #5849 from yhuai/serializer2MoreCases and squashes the following commits: 53a5eaa [Yin Huai] Josh's comments. 487f540 [Yin Huai] Use BufferedOutputStream. 8385f95 [Yin Huai] Always create a new row at the deserialization side to work with sort merge join. c7e2129 [Yin Huai] Update tests. 4513d13 [Yin Huai] Use Serializer2 in more places. (cherry picked from commit 3af423c92f117b5dd4dc6832dc50911cedb29abc) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9d0d2894 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9d0d2894 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9d0d2894 Branch: refs/heads/branch-1.4 Commit: 9d0d28940f54e90093db2794f16e9b1c9ce7a500 Parents: 28d4238 Author: Yin Huai yh...@databricks.com Authored: Thu May 7 20:59:42 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 20:59:53 2015 -0700 -- .../apache/spark/sql/execution/Exchange.scala | 23 ++ .../sql/execution/SparkSqlSerializer2.scala | 74 +--- .../execution/SparkSqlSerializer2Suite.scala| 30 3 files changed, 69 insertions(+), 58 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9d0d2894/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 5b2e469..f0d54cd 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 @@ -84,18 +84,8 @@ case class Exchange( def serializer( keySchema: Array[DataType], valueSchema: Array[DataType], + hasKeyOrdering: Boolean, numPartitions: Int): Serializer = { -// In ExternalSorter's spillToMergeableFile function, key-value pairs are written out -// through write(key) and then write(value) instead of write((key, value)). Because -// SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use -// it when spillToMergeableFile in ExternalSorter will be used. -// So, we will not use SparkSqlSerializer2 when -// - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater -// then the bypassMergeThreshold; or -// - newOrdering is defined. -val cannotUseSqlSerializer2 = - (sortBasedShuffleOn numPartitions bypassMergeThreshold) || newOrdering.nonEmpty - // It is true when there is no field that needs to be write out. // For now, we will not use SparkSqlSerializer2 when noField is true. val noField = @@ -104,14 +94,13 @@ case class Exchange( val useSqlSerializer2 = child.sqlContext.conf.useSqlSerializer2// SparkSqlSerializer2 is enabled. -!cannotUseSqlSerializer2 // Safe to use Serializer2. SparkSqlSerializer2.support(keySchema) // The schema of key is supported. SparkSqlSerializer2.support(valueSchema) // The schema of value is supported. !noField val serializer = if (useSqlSerializer2) { logInfo(Using SparkSqlSerializer2.) - new SparkSqlSerializer2(keySchema, valueSchema) + new SparkSqlSerializer2(keySchema, valueSchema, hasKeyOrdering) } else { logInfo(Using SparkSqlSerializer.) new SparkSqlSerializer(sparkConf) @@ -154,7 +143,8 @@ case class Exchange( } val keySchema = expressions.map(_.dataType).toArray val valueSchema = child.output.map(_.dataType).toArray -shuffled.setSerializer(serializer(keySchema, valueSchema, numPartitions)) +shuffled.setSerializer( + serializer(keySchema, valueSchema, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._2) @@ -179,7 +169,8 @@ case class Exchange( new ShuffledRDD[Row, Null, Null](rdd, part) } val keySchema = child.output.map(_.dataType).toArray -shuffled.setSerializer(serializer(keySchema, null, numPartitions)) +shuffled.setSerializer( + serializer(keySchema, null, newOrdering.nonEmpty, numPartitions)) shuffled.map
spark git commit: [SPARK-6986] [SQL] Use Serializer2 in more cases.
Repository: spark Updated Branches: refs/heads/master 92f8f803a - 3af423c92 [SPARK-6986] [SQL] Use Serializer2 in more cases. With https://github.com/apache/spark/commit/0a2b15ce43cf6096e1a7ae060b7c8a4010ce3b92, the serialization stream and deserialization stream has enough information to determine it is handling a key-value pari, a key, or a value. It is safe to use `SparkSqlSerializer2` in more cases. Author: Yin Huai yh...@databricks.com Closes #5849 from yhuai/serializer2MoreCases and squashes the following commits: 53a5eaa [Yin Huai] Josh's comments. 487f540 [Yin Huai] Use BufferedOutputStream. 8385f95 [Yin Huai] Always create a new row at the deserialization side to work with sort merge join. c7e2129 [Yin Huai] Update tests. 4513d13 [Yin Huai] Use Serializer2 in more places. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3af423c9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3af423c9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3af423c9 Branch: refs/heads/master Commit: 3af423c92f117b5dd4dc6832dc50911cedb29abc Parents: 92f8f80 Author: Yin Huai yh...@databricks.com Authored: Thu May 7 20:59:42 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 20:59:42 2015 -0700 -- .../apache/spark/sql/execution/Exchange.scala | 23 ++ .../sql/execution/SparkSqlSerializer2.scala | 74 +--- .../execution/SparkSqlSerializer2Suite.scala| 30 3 files changed, 69 insertions(+), 58 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3af423c9/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 5b2e469..f0d54cd 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 @@ -84,18 +84,8 @@ case class Exchange( def serializer( keySchema: Array[DataType], valueSchema: Array[DataType], + hasKeyOrdering: Boolean, numPartitions: Int): Serializer = { -// In ExternalSorter's spillToMergeableFile function, key-value pairs are written out -// through write(key) and then write(value) instead of write((key, value)). Because -// SparkSqlSerializer2 assumes that objects passed in are Product2, we cannot safely use -// it when spillToMergeableFile in ExternalSorter will be used. -// So, we will not use SparkSqlSerializer2 when -// - Sort-based shuffle is enabled and the number of reducers (numPartitions) is greater -// then the bypassMergeThreshold; or -// - newOrdering is defined. -val cannotUseSqlSerializer2 = - (sortBasedShuffleOn numPartitions bypassMergeThreshold) || newOrdering.nonEmpty - // It is true when there is no field that needs to be write out. // For now, we will not use SparkSqlSerializer2 when noField is true. val noField = @@ -104,14 +94,13 @@ case class Exchange( val useSqlSerializer2 = child.sqlContext.conf.useSqlSerializer2// SparkSqlSerializer2 is enabled. -!cannotUseSqlSerializer2 // Safe to use Serializer2. SparkSqlSerializer2.support(keySchema) // The schema of key is supported. SparkSqlSerializer2.support(valueSchema) // The schema of value is supported. !noField val serializer = if (useSqlSerializer2) { logInfo(Using SparkSqlSerializer2.) - new SparkSqlSerializer2(keySchema, valueSchema) + new SparkSqlSerializer2(keySchema, valueSchema, hasKeyOrdering) } else { logInfo(Using SparkSqlSerializer.) new SparkSqlSerializer(sparkConf) @@ -154,7 +143,8 @@ case class Exchange( } val keySchema = expressions.map(_.dataType).toArray val valueSchema = child.output.map(_.dataType).toArray -shuffled.setSerializer(serializer(keySchema, valueSchema, numPartitions)) +shuffled.setSerializer( + serializer(keySchema, valueSchema, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._2) @@ -179,7 +169,8 @@ case class Exchange( new ShuffledRDD[Row, Null, Null](rdd, part) } val keySchema = child.output.map(_.dataType).toArray -shuffled.setSerializer(serializer(keySchema, null, numPartitions)) +shuffled.setSerializer( + serializer(keySchema, null, newOrdering.nonEmpty, numPartitions)) shuffled.map(_._1) @@ -199,7 +190,7 @@ case class Exchange( val partitioner = new HashPartitioner(1) val shuffled
spark git commit: [SPARK-7470] [SQL] Spark shell SQLContext crashes without hive
Repository: spark Updated Branches: refs/heads/branch-1.4 9d0d28940 - 1a3e9e982 [SPARK-7470] [SQL] Spark shell SQLContext crashes without hive This only happens if you have `SPARK_PREPEND_CLASSES` set. Then I built it with `build/sbt clean assembly compile` and just ran it with `bin/spark-shell`. ``` ... 15/05/07 17:07:30 INFO EventLoggingListener: Logging events to file:/tmp/spark-events/local-1431043649919 15/05/07 17:07:30 INFO SparkILoop: Created spark context.. Spark context available as sc. java.lang.NoClassDefFoundError: org/apache/hadoop/hive/conf/HiveConf at java.lang.Class.getDeclaredConstructors0(Native Method) at java.lang.Class.privateGetDeclaredConstructors(Class.java:2493) at java.lang.Class.getConstructor0(Class.java:2803) ... Caused by: java.lang.ClassNotFoundException: org.apache.hadoop.hive.conf.HiveConf at java.net.URLClassLoader$1.run(URLClassLoader.java:366) at java.net.URLClassLoader$1.run(URLClassLoader.java:355) at java.security.AccessController.doPrivileged(Native Method) at java.net.URLClassLoader.findClass(URLClassLoader.java:354) at java.lang.ClassLoader.loadClass(ClassLoader.java:425) at sun.misc.Launcher$AppClassLoader.loadClass(Launcher.java:308) at java.lang.ClassLoader.loadClass(ClassLoader.java:358) ... 52 more console:10: error: not found: value sqlContext import sqlContext.implicits._ ^ console:10: error: not found: value sqlContext import sqlContext.sql ^ ``` yhuai marmbrus Author: Andrew Or and...@databricks.com Closes #5997 from andrewor14/sql-shell-crash and squashes the following commits: 61147e6 [Andrew Or] Also expect NoClassDefFoundError (cherry picked from commit 714db2ef52c0fe34418e252e5a6f220337022046) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/1a3e9e98 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/1a3e9e98 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/1a3e9e98 Branch: refs/heads/branch-1.4 Commit: 1a3e9e9828e12836a2600a62a8afefb23f7386d4 Parents: 9d0d289 Author: Andrew Or and...@databricks.com Authored: Thu May 7 22:32:13 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 22:32:42 2015 -0700 -- .../src/main/scala/org/apache/spark/repl/SparkILoop.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/1a3e9e98/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala -- diff --git a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala index 8dc0e0c..488f3a9 100644 --- a/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala +++ b/repl/scala-2.10/src/main/scala/org/apache/spark/repl/SparkILoop.scala @@ -1028,7 +1028,7 @@ class SparkILoop( logInfo(Created sql context (with Hive support)..) } catch { - case cnf: java.lang.ClassNotFoundException = + case _: java.lang.ClassNotFoundException | _: java.lang.NoClassDefFoundError = sqlContext = new SQLContext(sparkContext) logInfo(Created sql context..) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [HOT-FIX] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir.
Repository: spark Updated Branches: refs/heads/master 845d1d4d0 - 774099670 [HOT-FIX] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir. Author: Yin Huai yh...@databricks.com Closes #5951 from yhuai/fixBuildMaven and squashes the following commits: fdde183 [Yin Huai] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/77409967 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/77409967 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/77409967 Branch: refs/heads/master Commit: 77409967008415b7ea57e9349d296350e6519687 Parents: 845d1d4 Author: Yin Huai yh...@databricks.com Authored: Wed May 6 14:48:25 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 6 14:48:25 2015 -0700 -- .../HiveWindowFunctionQuerySuite.scala | 845 +++ .../HiveWindowFunctionQuerySuite.scala | 845 --- 2 files changed, 845 insertions(+), 845 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/77409967/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala -- diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala new file mode 100644 index 000..934452f --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -0,0 +1,845 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the License); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + +/** + * The test suite for window functions. To actually compare results with Hive, + * every test should be created by `createQueryTest`. Because we are reusing tables + * for different tests and there are a few properties needed to let Hive generate golden + * files, every `createQueryTest` calls should explicitly set `reset` to `false`. + */ +abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { +TestHive.cacheTables = true +// Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) +TimeZone.setDefault(TimeZone.getTimeZone(America/Los_Angeles)) +// Add Locale setting +Locale.setDefault(Locale.US) + +// Create the table used in windowing.q +sql(DROP TABLE IF EXISTS part) +sql( + +|CREATE TABLE part( +| p_partkey INT, +| p_name STRING, +| p_mfgr STRING, +| p_brand STRING, +| p_type STRING, +| p_size INT, +| p_container STRING, +| p_retailprice DOUBLE, +| p_comment STRING) + .stripMargin) +val testData1 = TestHive.getHiveFile(data/files/part_tiny.txt).getCanonicalPath +sql( + s +|LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + .stripMargin) + +sql(DROP TABLE IF EXISTS over1k) +sql( + +|create table over1k( +| t tinyint, +| si smallint, +| i int, +| b bigint, +| f float, +| d double, +| bo boolean, +| s string, +| ts timestamp, +| dec decimal(4,2), +| bin binary) +|row format delimited +|fields terminated by '|' + .stripMargin) +val testData2 = TestHive.getHiveFile(data/files/over1k
spark git commit: [HOT-FIX] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir.
Repository: spark Updated Branches: refs/heads/branch-1.4 2163367ea - 14bcb84e8 [HOT-FIX] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir. Author: Yin Huai yh...@databricks.com Closes #5951 from yhuai/fixBuildMaven and squashes the following commits: fdde183 [Yin Huai] Move HiveWindowFunctionQuerySuite.scala to hive compatibility dir. (cherry picked from commit 77409967008415b7ea57e9349d296350e6519687) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/14bcb84e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/14bcb84e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/14bcb84e Branch: refs/heads/branch-1.4 Commit: 14bcb84e84be9d9cc847b4b9d5d803f5ecb89bc5 Parents: 2163367 Author: Yin Huai yh...@databricks.com Authored: Wed May 6 14:48:25 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 6 14:49:13 2015 -0700 -- .../HiveWindowFunctionQuerySuite.scala | 845 +++ .../HiveWindowFunctionQuerySuite.scala | 845 --- 2 files changed, 845 insertions(+), 845 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/14bcb84e/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala -- diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala new file mode 100644 index 000..934452f --- /dev/null +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -0,0 +1,845 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the License); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.hive.execution + +import java.io.File +import java.util.{Locale, TimeZone} + +import org.scalatest.BeforeAndAfter + +import org.apache.spark.sql.hive.test.TestHive +import org.apache.spark.sql.hive.test.TestHive._ +import org.apache.spark.util.Utils + +/** + * The test suite for window functions. To actually compare results with Hive, + * every test should be created by `createQueryTest`. Because we are reusing tables + * for different tests and there are a few properties needed to let Hive generate golden + * files, every `createQueryTest` calls should explicitly set `reset` to `false`. + */ +abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { + private val originalTimeZone = TimeZone.getDefault + private val originalLocale = Locale.getDefault + private val testTempDir = Utils.createTempDir() + + override def beforeAll() { +TestHive.cacheTables = true +// Timezone is fixed to America/Los_Angeles for those timezone sensitive tests (timestamp_*) +TimeZone.setDefault(TimeZone.getTimeZone(America/Los_Angeles)) +// Add Locale setting +Locale.setDefault(Locale.US) + +// Create the table used in windowing.q +sql(DROP TABLE IF EXISTS part) +sql( + +|CREATE TABLE part( +| p_partkey INT, +| p_name STRING, +| p_mfgr STRING, +| p_brand STRING, +| p_type STRING, +| p_size INT, +| p_container STRING, +| p_retailprice DOUBLE, +| p_comment STRING) + .stripMargin) +val testData1 = TestHive.getHiveFile(data/files/part_tiny.txt).getCanonicalPath +sql( + s +|LOAD DATA LOCAL INPATH '$testData1' overwrite into table part + .stripMargin) + +sql(DROP TABLE IF EXISTS over1k) +sql( + +|create table over1k( +| t tinyint, +| si smallint, +| i int, +| b bigint, +| f float, +| d double, +| bo boolean, +| s string, +| ts timestamp, +| dec decimal(4,2), +| bin binary) +|row format delimited
spark git commit: [SPARK-6201] [SQL] promote string and do widen types for IN
Repository: spark Updated Branches: refs/heads/master 150f671c2 - c3eb441f5 [SPARK-6201] [SQL] promote string and do widen types for IN huangjs Acutally spark sql will first go through analysis period, in which we do widen types and promote strings, and then optimization, where constant IN will be converted into INSET. So it turn out that we only need to fix this for IN. Author: Daoyuan Wang daoyuan.w...@intel.com Closes #4945 from adrian-wang/inset and squashes the following commits: 71e05cc [Daoyuan Wang] minor fix 581fa1c [Daoyuan Wang] mysql way f3f7baf [Daoyuan Wang] address comments 5eed4bc [Daoyuan Wang] promote string and do widen types for IN Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c3eb441f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c3eb441f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c3eb441f Branch: refs/heads/master Commit: c3eb441f5487c9b6476e1d6e2a2d852dcc43b986 Parents: 150f671 Author: Daoyuan Wang daoyuan.w...@intel.com Authored: Wed May 6 10:30:42 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 6 10:30:42 2015 -0700 -- .../spark/sql/catalyst/analysis/HiveTypeCoercion.scala | 11 +++ .../apache/spark/sql/catalyst/optimizer/Optimizer.scala | 4 ++-- .../test/scala/org/apache/spark/sql/SQLQuerySuite.scala | 9 + 3 files changed, 22 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c3eb441f/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 831fb4f..96e2aee 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 @@ -69,6 +69,7 @@ trait HiveTypeCoercion { val typeCoercionRules = PropagateTypes :: ConvertNaNs :: +InConversion :: WidenTypes :: PromoteStrings :: DecimalPrecision :: @@ -287,6 +288,16 @@ trait HiveTypeCoercion { } } + /** + * Convert all expressions in in() list to the left operator type + */ + object InConversion extends Rule[LogicalPlan] { +def apply(plan: LogicalPlan): LogicalPlan = plan transformAllExpressions { + case i @ In(a, b) if b.exists(_.dataType != a.dataType) = +i.makeCopy(Array(a, b.map(Cast(_, a.dataType +} + } + // scalastyle:off /** * Calculates and propagates precision for fixed-precision decimals. Hive has a number of http://git-wip-us.apache.org/repos/asf/spark/blob/c3eb441f/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 709f7d6..e4a60f5 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 @@ -310,8 +310,8 @@ object OptimizeIn extends Rule[LogicalPlan] { def apply(plan: LogicalPlan): LogicalPlan = plan transform { case q: LogicalPlan = q transformExpressionsDown { case In(v, list) if !list.exists(!_.isInstanceOf[Literal]) = - val hSet = list.map(e = e.eval(null)) - InSet(v, HashSet() ++ hSet) +val hSet = list.map(e = e.eval(null)) +InSet(v, HashSet() ++ hSet) } } } http://git-wip-us.apache.org/repos/asf/spark/blob/c3eb441f/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala index 0ab8558..208cec6 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala @@ -120,6 +120,15 @@ class SQLQuerySuite extends QueryTest with BeforeAndAfterAll { Row(1, 1) :: Nil) } + test(SPARK-6201 IN type conversion) { +jsonRDD(sparkContext.parallelize(Seq({\a\: \1\}}, {\a\: \2\}}, {\a\: \3\}}))) + .registerTempTable(d) + +checkAnswer( + sql(select * from d where d.a in (1,2)), + Seq(Row(1), Row(2))) + } + test(SPARK-3176 Added Parser of SQL ABS()) { checkAnswer(
spark git commit: [SPARK-7375] [SQL] Avoid row copying in exchange when sort.serializeMapOutputs takes effect
Repository: spark Updated Branches: refs/heads/master 0a901dd3a - cde548388 [SPARK-7375] [SQL] Avoid row copying in exchange when sort.serializeMapOutputs takes effect This patch refactors the SQL `Exchange` operator's logic for determining whether map outputs need to be copied before being shuffled. As part of this change, we'll now avoid unnecessary copies in cases where sort-based shuffle operates on serialized map outputs (as in #4450 / SPARK-4550). This patch also includes a change to copy the input to RangePartitioner partition bounds calculation, which is necessary because this calculation buffers mutable Java objects. !-- Reviewable:start -- [img src=https://reviewable.io/review_button.png; height=40 alt=Review on Reviewable/](https://reviewable.io/reviews/apache/spark/5948) !-- Reviewable:end -- Author: Josh Rosen joshro...@databricks.com Closes #5948 from JoshRosen/SPARK-7375 and squashes the following commits: f305ff3 [Josh Rosen] Reduce scope of some variables in Exchange 899e1d7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-7375 6a6bfce [Josh Rosen] Fix issue related to RangePartitioning: ad006a4 [Josh Rosen] [SPARK-7375] Avoid defensive copying in exchange operator when sort.serializeMapOutputs takes effect. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cde54838 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cde54838 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cde54838 Branch: refs/heads/master Commit: cde5483884068b0ae1470b9b9b3ee54ab944ab12 Parents: 0a901dd Author: Josh Rosen joshro...@databricks.com Authored: Fri May 8 22:09:55 2015 -0400 Committer: Yin Huai yh...@databricks.com Committed: Fri May 8 22:09:55 2015 -0400 -- .../apache/spark/sql/execution/Exchange.scala | 156 --- 1 file changed, 100 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cde54838/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 f02fa81..c3d2c70 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner} +import org.apache.spark.{HashPartitioner, Partitioner, RangePartitioner, SparkEnv} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.serializer.Serializer import org.apache.spark.sql.{SQLContext, Row} @@ -59,11 +59,62 @@ case class Exchange( override def output: Seq[Attribute] = child.output - /** We must copy rows when sort based shuffle is on */ - protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] - - private val bypassMergeThreshold = - child.sqlContext.sparkContext.conf.getInt(spark.shuffle.sort.bypassMergeThreshold, 200) + /** + * Determines whether records must be defensively copied before being sent to the shuffle. + * Several of Spark's shuffle components will buffer deserialized Java objects in memory. The + * shuffle code assumes that objects are immutable and hence does not perform its own defensive + * copying. In Spark SQL, however, operators' iterators return the same mutable `Row` object. In + * order to properly shuffle the output of these operators, we need to perform our own copying + * prior to sending records to the shuffle. This copying is expensive, so we try to avoid it + * whenever possible. This method encapsulates the logic for choosing when to copy. + * + * In the long run, we might want to push this logic into core's shuffle APIs so that we don't + * have to rely on knowledge of core internals here in SQL. + * + * See SPARK-2967, SPARK-4479, and SPARK-7375 for more discussion of this issue. + * + * @param partitioner the partitioner for the shuffle + * @param serializer the serializer that will be used to write rows + * @return true if rows should be copied before being shuffled, false otherwise + */ + private def needToCopyObjectsBeforeShuffle( + partitioner: Partitioner, + serializer: Serializer): Boolean = { +// Note: even though we only use the partitioner's `numPartitions` field, we require it to be +// passed instead of directly passing the number of partitions in order to guard against +// corner-cases where a partitioner
spark git commit: [SPARK-7375] [SQL] Avoid row copying in exchange when sort.serializeMapOutputs takes effect
Repository: spark Updated Branches: refs/heads/branch-1.4 448ff333f - 21212a27c [SPARK-7375] [SQL] Avoid row copying in exchange when sort.serializeMapOutputs takes effect This patch refactors the SQL `Exchange` operator's logic for determining whether map outputs need to be copied before being shuffled. As part of this change, we'll now avoid unnecessary copies in cases where sort-based shuffle operates on serialized map outputs (as in #4450 / SPARK-4550). This patch also includes a change to copy the input to RangePartitioner partition bounds calculation, which is necessary because this calculation buffers mutable Java objects. !-- Reviewable:start -- [img src=https://reviewable.io/review_button.png; height=40 alt=Review on Reviewable/](https://reviewable.io/reviews/apache/spark/5948) !-- Reviewable:end -- Author: Josh Rosen joshro...@databricks.com Closes #5948 from JoshRosen/SPARK-7375 and squashes the following commits: f305ff3 [Josh Rosen] Reduce scope of some variables in Exchange 899e1d7 [Josh Rosen] Merge remote-tracking branch 'origin/master' into SPARK-7375 6a6bfce [Josh Rosen] Fix issue related to RangePartitioning: ad006a4 [Josh Rosen] [SPARK-7375] Avoid defensive copying in exchange operator when sort.serializeMapOutputs takes effect. (cherry picked from commit cde5483884068b0ae1470b9b9b3ee54ab944ab12) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/21212a27 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/21212a27 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/21212a27 Branch: refs/heads/branch-1.4 Commit: 21212a27cbcfa240d0cbb93ce81883409904ffd4 Parents: 448ff33 Author: Josh Rosen joshro...@databricks.com Authored: Fri May 8 22:09:55 2015 -0400 Committer: Yin Huai yh...@databricks.com Committed: Fri May 8 22:10:10 2015 -0400 -- .../apache/spark/sql/execution/Exchange.scala | 156 --- 1 file changed, 100 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/21212a27/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 f02fa81..c3d2c70 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 @@ -19,7 +19,7 @@ package org.apache.spark.sql.execution import org.apache.spark.annotation.DeveloperApi import org.apache.spark.shuffle.sort.SortShuffleManager -import org.apache.spark.{SparkEnv, HashPartitioner, RangePartitioner} +import org.apache.spark.{HashPartitioner, Partitioner, RangePartitioner, SparkEnv} import org.apache.spark.rdd.{RDD, ShuffledRDD} import org.apache.spark.serializer.Serializer import org.apache.spark.sql.{SQLContext, Row} @@ -59,11 +59,62 @@ case class Exchange( override def output: Seq[Attribute] = child.output - /** We must copy rows when sort based shuffle is on */ - protected def sortBasedShuffleOn = SparkEnv.get.shuffleManager.isInstanceOf[SortShuffleManager] - - private val bypassMergeThreshold = - child.sqlContext.sparkContext.conf.getInt(spark.shuffle.sort.bypassMergeThreshold, 200) + /** + * Determines whether records must be defensively copied before being sent to the shuffle. + * Several of Spark's shuffle components will buffer deserialized Java objects in memory. The + * shuffle code assumes that objects are immutable and hence does not perform its own defensive + * copying. In Spark SQL, however, operators' iterators return the same mutable `Row` object. In + * order to properly shuffle the output of these operators, we need to perform our own copying + * prior to sending records to the shuffle. This copying is expensive, so we try to avoid it + * whenever possible. This method encapsulates the logic for choosing when to copy. + * + * In the long run, we might want to push this logic into core's shuffle APIs so that we don't + * have to rely on knowledge of core internals here in SQL. + * + * See SPARK-2967, SPARK-4479, and SPARK-7375 for more discussion of this issue. + * + * @param partitioner the partitioner for the shuffle + * @param serializer the serializer that will be used to write rows + * @return true if rows should be copied before being shuffled, false otherwise + */ + private def needToCopyObjectsBeforeShuffle( + partitioner: Partitioner, + serializer: Serializer): Boolean = { +// Note: even though we only use the partitioner's `numPartitions` field, we require it to be +//
spark git commit: [SPARK-7330] [SQL] avoid NPE at jdbc rdd
Repository: spark Updated Branches: refs/heads/master 4f87e9562 - ed9be06a4 [SPARK-7330] [SQL] avoid NPE at jdbc rdd Thank nadavoosh point this out in #5590 Author: Daoyuan Wang daoyuan.w...@intel.com Closes #5877 from adrian-wang/jdbcrdd and squashes the following commits: cc11900 [Daoyuan Wang] avoid NPE in jdbcrdd Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ed9be06a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ed9be06a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ed9be06a Branch: refs/heads/master Commit: ed9be06a4797bbb678355b361054c8872ac20b75 Parents: 4f87e95 Author: Daoyuan Wang daoyuan.w...@intel.com Authored: Thu May 7 10:05:01 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 10:05:01 2015 -0700 -- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 8 ++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 25 2 files changed, 32 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ed9be06a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 325a326..1a5083d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -362,7 +362,13 @@ private[sql] class JDBCRDD( conversions(i) match { case BooleanConversion= mutableRow.setBoolean(i, rs.getBoolean(pos)) case DateConversion = - mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) + // DateUtils.fromJavaDate does not handle null value, so we need to check it. + val dateVal = rs.getDate(pos) + if (dateVal != null) { +mutableRow.update(i, DateUtils.fromJavaDate(dateVal)) + } else { +mutableRow.update(i, null) + } case DecimalConversion= val decimalVal = rs.getBigDecimal(pos) if (decimalVal == null) { http://git-wip-us.apache.org/repos/asf/spark/blob/ed9be06a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 3ec17d3..021affa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -104,6 +104,8 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { ).executeUpdate() conn.prepareStatement(insert into test.timetypes values ('12:34:56', + '1996-01-01', '2002-02-20 11:22:33.543543543')).executeUpdate() +conn.prepareStatement(insert into test.timetypes values ('12:34:56', + + null, '2002-02-20 11:22:33.543543543')).executeUpdate() conn.commit() sql( s @@ -127,6 +129,23 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') .stripMargin.replaceAll(\n, )) +conn.prepareStatement( + s +|create table test.nulltypes (a INT, b BOOLEAN, c TINYINT, d BINARY(20), e VARCHAR(20), +|f VARCHAR_IGNORECASE(20), g CHAR(20), h BLOB, i CLOB, j TIME, k DATE, l TIMESTAMP, +|m DOUBLE, n REAL, o DECIMAL(40, 20)) + .stripMargin.replaceAll(\n, )).executeUpdate() +conn.prepareStatement(insert into test.nulltypes values ( + + null, null, null, null, null, null, null, null, null, + + null, null, null, null, null, null)).executeUpdate() +conn.commit() +sql( + s + |CREATE TEMPORARY TABLE nulltypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.NULLTYPES', user 'testUser', password 'testPass') + .stripMargin.replaceAll(\n, )) + // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } @@ -254,6 +273,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { val rows = TestSQLContext.jdbc(urlWithUserAndPass, TEST.TIMETYPES).collect() val cachedRows = TestSQLContext.jdbc(urlWithUserAndPass, TEST.TIMETYPES).cache().collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf(1996-01-01)) +assert(rows(1).getAs[java.sql.Date](1) === null) assert(cachedRows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf(1996-01-01)) } @@ -266,6 +286,11 @@ class JDBCSuite extends
spark git commit: [SPARK-7330] [SQL] avoid NPE at jdbc rdd
Repository: spark Updated Branches: refs/heads/branch-1.4 91ce13109 - 84ee348bc [SPARK-7330] [SQL] avoid NPE at jdbc rdd Thank nadavoosh point this out in #5590 Author: Daoyuan Wang daoyuan.w...@intel.com Closes #5877 from adrian-wang/jdbcrdd and squashes the following commits: cc11900 [Daoyuan Wang] avoid NPE in jdbcrdd (cherry picked from commit ed9be06a4797bbb678355b361054c8872ac20b75) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/84ee348b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/84ee348b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/84ee348b Branch: refs/heads/branch-1.4 Commit: 84ee348bce9f06626d4ca80358ae3eed32b4dd36 Parents: 91ce131 Author: Daoyuan Wang daoyuan.w...@intel.com Authored: Thu May 7 10:05:01 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 10:05:19 2015 -0700 -- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 8 ++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 25 2 files changed, 32 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/84ee348b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index 325a326..1a5083d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -362,7 +362,13 @@ private[sql] class JDBCRDD( conversions(i) match { case BooleanConversion= mutableRow.setBoolean(i, rs.getBoolean(pos)) case DateConversion = - mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) + // DateUtils.fromJavaDate does not handle null value, so we need to check it. + val dateVal = rs.getDate(pos) + if (dateVal != null) { +mutableRow.update(i, DateUtils.fromJavaDate(dateVal)) + } else { +mutableRow.update(i, null) + } case DecimalConversion= val decimalVal = rs.getBigDecimal(pos) if (decimalVal == null) { http://git-wip-us.apache.org/repos/asf/spark/blob/84ee348b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index 3ec17d3..021affa 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -104,6 +104,8 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { ).executeUpdate() conn.prepareStatement(insert into test.timetypes values ('12:34:56', + '1996-01-01', '2002-02-20 11:22:33.543543543')).executeUpdate() +conn.prepareStatement(insert into test.timetypes values ('12:34:56', + + null, '2002-02-20 11:22:33.543543543')).executeUpdate() conn.commit() sql( s @@ -127,6 +129,23 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') .stripMargin.replaceAll(\n, )) +conn.prepareStatement( + s +|create table test.nulltypes (a INT, b BOOLEAN, c TINYINT, d BINARY(20), e VARCHAR(20), +|f VARCHAR_IGNORECASE(20), g CHAR(20), h BLOB, i CLOB, j TIME, k DATE, l TIMESTAMP, +|m DOUBLE, n REAL, o DECIMAL(40, 20)) + .stripMargin.replaceAll(\n, )).executeUpdate() +conn.prepareStatement(insert into test.nulltypes values ( + + null, null, null, null, null, null, null, null, null, + + null, null, null, null, null, null)).executeUpdate() +conn.commit() +sql( + s + |CREATE TEMPORARY TABLE nulltypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.NULLTYPES', user 'testUser', password 'testPass') + .stripMargin.replaceAll(\n, )) + // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } @@ -254,6 +273,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { val rows = TestSQLContext.jdbc(urlWithUserAndPass, TEST.TIMETYPES).collect() val cachedRows = TestSQLContext.jdbc(urlWithUserAndPass, TEST.TIMETYPES).cache().collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf(1996-01-01)) +assert(rows(1).getAs[java.sql.Date](1) === null)
spark git commit: [SPARK-7330] [SQL] avoid NPE at jdbc rdd
Repository: spark Updated Branches: refs/heads/branch-1.3 cbf232daa - edcd3643a [SPARK-7330] [SQL] avoid NPE at jdbc rdd Thank nadavoosh point this out in #5590 Author: Daoyuan Wang daoyuan.w...@intel.com Closes #5877 from adrian-wang/jdbcrdd and squashes the following commits: cc11900 [Daoyuan Wang] avoid NPE in jdbcrdd (cherry picked from commit ed9be06a4797bbb678355b361054c8872ac20b75) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/edcd3643 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/edcd3643 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/edcd3643 Branch: refs/heads/branch-1.3 Commit: edcd3643aa2bc39571dce17b7517eb40ced2a7f5 Parents: cbf232d Author: Daoyuan Wang daoyuan.w...@intel.com Authored: Thu May 7 10:05:01 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 7 10:17:58 2015 -0700 -- .../org/apache/spark/sql/jdbc/JDBCRDD.scala | 8 ++- .../org/apache/spark/sql/jdbc/JDBCSuite.scala | 25 2 files changed, 32 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/edcd3643/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala index d58b686..2881d9a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/jdbc/JDBCRDD.scala @@ -357,7 +357,13 @@ private[sql] class JDBCRDD( conversions(i) match { case BooleanConversion= mutableRow.setBoolean(i, rs.getBoolean(pos)) case DateConversion = - mutableRow.update(i, DateUtils.fromJavaDate(rs.getDate(pos))) + // DateUtils.fromJavaDate does not handle null value, so we need to check it. + val dateVal = rs.getDate(pos) + if (dateVal != null) { +mutableRow.update(i, DateUtils.fromJavaDate(dateVal)) + } else { +mutableRow.update(i, null) + } case DecimalConversion= val decimalVal = rs.getBigDecimal(pos) if (decimalVal == null) { http://git-wip-us.apache.org/repos/asf/spark/blob/edcd3643/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala index e8f7932..7719a76 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/jdbc/JDBCSuite.scala @@ -102,6 +102,8 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { ).executeUpdate() conn.prepareStatement(insert into test.timetypes values ('12:34:56', + '1996-01-01', '2002-02-20 11:22:33.543543543')).executeUpdate() +conn.prepareStatement(insert into test.timetypes values ('12:34:56', + + null, '2002-02-20 11:22:33.543543543')).executeUpdate() conn.commit() sql( s @@ -125,6 +127,23 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { |OPTIONS (url '$url', dbtable 'TEST.FLTTYPES', user 'testUser', password 'testPass') .stripMargin.replaceAll(\n, )) +conn.prepareStatement( + s +|create table test.nulltypes (a INT, b BOOLEAN, c TINYINT, d BINARY(20), e VARCHAR(20), +|f VARCHAR_IGNORECASE(20), g CHAR(20), h BLOB, i CLOB, j TIME, k DATE, l TIMESTAMP, +|m DOUBLE, n REAL, o DECIMAL(40, 20)) + .stripMargin.replaceAll(\n, )).executeUpdate() +conn.prepareStatement(insert into test.nulltypes values ( + + null, null, null, null, null, null, null, null, null, + + null, null, null, null, null, null)).executeUpdate() +conn.commit() +sql( + s + |CREATE TEMPORARY TABLE nulltypes + |USING org.apache.spark.sql.jdbc + |OPTIONS (url '$url', dbtable 'TEST.NULLTYPES', user 'testUser', password 'testPass') + .stripMargin.replaceAll(\n, )) + // Untested: IDENTITY, OTHER, UUID, ARRAY, and GEOMETRY types. } @@ -252,6 +271,7 @@ class JDBCSuite extends FunSuite with BeforeAndAfter { val rows = TestSQLContext.jdbc(urlWithUserAndPass, TEST.TIMETYPES).collect() val cachedRows = TestSQLContext.jdbc(urlWithUserAndPass, TEST.TIMETYPES).cache().collect() assert(rows(0).getAs[java.sql.Date](1) === java.sql.Date.valueOf(1996-01-01)) +assert(rows(1).getAs[java.sql.Date](1) === null)
spark git commit: [SQL] [MINOR] use catalyst type converter in ScalaUdf
Repository: spark Updated Branches: refs/heads/branch-1.4 e0632ffaf - be66d1924 [SQL] [MINOR] use catalyst type converter in ScalaUdf It's a follow-up of https://github.com/apache/spark/pull/5154, we can speed up scala udf evaluation by create type converter in advance. Author: Wenchen Fan cloud0...@outlook.com Closes #6182 from cloud-fan/tmp and squashes the following commits: 241cfe9 [Wenchen Fan] use converter in ScalaUdf (cherry picked from commit 2f22424e9f6624097b292cb70e00787b69d80718) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/be66d192 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/be66d192 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/be66d192 Branch: refs/heads/branch-1.4 Commit: be66d1924edc5c99987c80d445f34a690c3789a9 Parents: e0632ff Author: Wenchen Fan cloud0...@outlook.com Authored: Sun May 17 16:51:57 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun May 17 16:52:21 2015 -0700 -- .../org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/be66d192/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 9a77ca6..d22eb10 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 @@ -956,7 +956,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi } // scalastyle:on - - override def eval(input: Row): Any = CatalystTypeConverters.convertToCatalyst(f(input), dataType) + val converter = CatalystTypeConverters.createToCatalystConverter(dataType) + override def eval(input: Row): Any = converter(f(input)) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SQL] [MINOR] use catalyst type converter in ScalaUdf
Repository: spark Updated Branches: refs/heads/master ca4257aec - 2f22424e9 [SQL] [MINOR] use catalyst type converter in ScalaUdf It's a follow-up of https://github.com/apache/spark/pull/5154, we can speed up scala udf evaluation by create type converter in advance. Author: Wenchen Fan cloud0...@outlook.com Closes #6182 from cloud-fan/tmp and squashes the following commits: 241cfe9 [Wenchen Fan] use converter in ScalaUdf Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2f22424e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2f22424e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2f22424e Branch: refs/heads/master Commit: 2f22424e9f6624097b292cb70e00787b69d80718 Parents: ca4257a Author: Wenchen Fan cloud0...@outlook.com Authored: Sun May 17 16:51:57 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun May 17 16:51:57 2015 -0700 -- .../org/apache/spark/sql/catalyst/expressions/ScalaUdf.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2f22424e/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 9a77ca6..d22eb10 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 @@ -956,7 +956,7 @@ case class ScalaUdf(function: AnyRef, dataType: DataType, children: Seq[Expressi } // scalastyle:on - - override def eval(input: Row): Any = CatalystTypeConverters.convertToCatalyst(f(input), dataType) + val converter = CatalystTypeConverters.createToCatalystConverter(dataType) + override def eval(input: Row): Any = converter(f(input)) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7673] [SQL] WIP: HadoopFsRelation and ParquetRelation2 performance optimizations
Repository: spark Updated Branches: refs/heads/master 530397ba2 - 9dadf019b [SPARK-7673] [SQL] WIP: HadoopFsRelation and ParquetRelation2 performance optimizations This PR introduces several performance optimizations to `HadoopFsRelation` and `ParquetRelation2`: 1. Moving `FileStatus` listing from `DataSourceStrategy` into a cache within `HadoopFsRelation`. This new cache generalizes and replaces the one used in `ParquetRelation2`. This also introduces an interface change: to reuse cached `FileStatus` objects, `HadoopFsRelation.buildScan` methods now receive `Array[FileStatus]` instead of `Array[String]`. 1. When Parquet task side metadata reading is enabled, skip reading row group information when reading Parquet footers. This is basically what PR #5334 does. Also, now we uses `ParquetFileReader.readAllFootersInParallel` to read footers in parallel. Another optimization in question is, instead of asking `HadoopFsRelation.buildScan` to return an `RDD[Row]` for a single selected partition and then union them all, we ask it to return an `RDD[Row]` for all selected partitions. This optimization is based on the fact that Hadoop configuration broadcasting used in `NewHadoopRDD` takes 34% time in the following microbenchmark. However, this complicates data source user code because user code must merge partition values manually. To check the cost of broadcasting in `NewHadoopRDD`, I also did microbenchmark after removing the `broadcast` call in `NewHadoopRDD`. All results are shown below. ### Microbenchmark Preparation code Generating a partitioned table with 50k partitions, 1k rows per partition: ```scala import sqlContext._ import sqlContext.implicits._ for (n - 0 until 500) { val data = for { p - (n * 10) until ((n + 1) * 10) i - 0 until 1000 } yield (i, fval_$i%04d, f$p%04d) data. toDF(a, b, p). write. partitionBy(p). mode(append). parquet(path) } ``` Benchmarking code ```scala import sqlContext._ import sqlContext.implicits._ import org.apache.spark.sql.types._ import com.google.common.base.Stopwatch val path = hdfs://localhost:9000/user/lian/5k def benchmark(n: Int)(f: = Unit) { val stopwatch = new Stopwatch() def run() = { stopwatch.reset() stopwatch.start() f stopwatch.stop() stopwatch.elapsedMillis() } val records = (0 until n).map(_ = run()) (0 until n).foreach(i = println(sRound $i: ${records(i)} ms)) println(sAverage: ${records.sum / n.toDouble} ms) } benchmark(3) { read.parquet(path).explain(extended = true) } ``` Results Before: ``` Round 0: 72528 ms Round 1: 68938 ms Round 2: 65372 ms Average: 68946.0 ms ``` After: ``` Round 0: 59499 ms Round 1: 53645 ms Round 2: 53844 ms Round 3: 49093 ms Round 4: 50555 ms Average: 53327.2 ms ``` Also removing Hadoop configuration broadcasting: (Note that I was testing on a local laptop, thus network cost is pretty low.) ``` Round 0: 15806 ms Round 1: 14394 ms Round 2: 14699 ms Round 3: 15334 ms Round 4: 14123 ms Average: 14871.2 ms ``` Author: Cheng Lian l...@databricks.com Closes #6225 from liancheng/spark-7673 and squashes the following commits: 2d58a2b [Cheng Lian] Skips reading row group information when using task side metadata reading 7aa3748 [Cheng Lian] Optimizes FileStatusCache by introducing a map from parent directories to child files ba41250 [Cheng Lian] Reuses HadoopFsRelation FileStatusCache in ParquetRelation2 3d278f7 [Cheng Lian] Fixes a bug when reading a single Parquet data file b84612a [Cheng Lian] Fixes Scala style issue 6a08b02 [Cheng Lian] WIP: Moves file status cache into HadoopFSRelation Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9dadf019 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9dadf019 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9dadf019 Branch: refs/heads/master Commit: 9dadf019b93038e1e18336ccd06c5eecb4bae32f Parents: 530397b Author: Cheng Lian l...@databricks.com Authored: Mon May 18 12:45:37 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon May 18 12:45:37 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 61 +-- .../spark/sql/sources/DataSourceStrategy.scala | 37 ++- .../apache/spark/sql/sources/interfaces.scala | 104 +++ .../spark/sql/sources/SimpleTextRelation.scala | 6 +- 4 files changed, 117 insertions(+), 91 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9dadf019/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala
spark git commit: [SPARK-7713] [SQL] Use shared broadcast hadoop conf for partitioned table scan.
Repository: spark Updated Branches: refs/heads/master 98a46f9df - b631bf73b [SPARK-7713] [SQL] Use shared broadcast hadoop conf for partitioned table scan. https://issues.apache.org/jira/browse/SPARK-7713 I tested the performance with the following code: ```scala import sqlContext._ import sqlContext.implicits._ (1 to 5000).foreach { i = val df = (1 to 1000).map(j = (j, sstr$j)).toDF(a, b).save(s/tmp/partitioned/i=$i) } sqlContext.sql( CREATE TEMPORARY TABLE partitionedParquet USING org.apache.spark.sql.parquet OPTIONS ( path '/tmp/partitioned' )) table(partitionedParquet).explain(true) ``` In our master `explain` takes 40s in my laptop. With this PR, `explain` takes 14s. Author: Yin Huai yh...@databricks.com Closes #6252 from yhuai/broadcastHadoopConf and squashes the following commits: 6fa73df [Yin Huai] Address comments of Josh and Andrew. 807fbf9 [Yin Huai] Make the new buildScan and SqlNewHadoopRDD private sql. e393555 [Yin Huai] Cheng's comments. 2eb53bb [Yin Huai] Use a shared broadcast Hadoop Configuration for partitioned HadoopFsRelations. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b631bf73 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b631bf73 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b631bf73 Branch: refs/heads/master Commit: b631bf73b9f288f37c98b806be430b22485880e5 Parents: 98a46f9 Author: Yin Huai yh...@databricks.com Authored: Wed May 20 11:23:40 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 20 11:23:40 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 113 +--- .../spark/sql/sources/DataSourceStrategy.scala | 19 +- .../spark/sql/sources/SqlNewHadoopRDD.scala | 268 +++ .../apache/spark/sql/sources/interfaces.scala | 35 ++- 4 files changed, 387 insertions(+), 48 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b631bf73/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 7ca44f7..c35b7ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConversions._ import scala.util.Try import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -32,13 +33,14 @@ import parquet.hadoop._ import parquet.hadoop.metadata.CompressionCodecName import parquet.hadoop.util.ContextUtil +import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD._ -import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.{Row, SQLConf, SQLContext} -import org.apache.spark.{Logging, Partition = SparkPartition, SparkException} +import org.apache.spark.{Partition = SparkPartition, SparkEnv, SerializableWritable, Logging, SparkException} private[sql] class DefaultSource extends HadoopFsRelationProvider { override def createRelation( @@ -233,40 +235,20 @@ private[sql] class ParquetRelation2( override def buildScan( requiredColumns: Array[String], filters: Array[Filter], - inputFiles: Array[FileStatus]): RDD[Row] = { - -val job = new Job(SparkHadoopUtil.get.conf) -val conf = ContextUtil.getConfiguration(job) - -ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) - -if (inputFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) -} - -// Try to push down filters when filter push-down is enabled. -if (sqlContext.conf.parquetFilterPushDown) { - filters -// Collects all converted Parquet filter predicates. Notice that not all predicates can be -// converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` -// is used here. -.flatMap(ParquetFilters.createFilter(dataSchema, _)) -.reduceOption(FilterApi.and) -.foreach(ParquetInputFormat.setFilterPredicate(conf, _)) -} - -conf.set(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { - val requestedSchema = StructType(requiredColumns.map(dataSchema(_))) - ParquetTypesConverter.convertToString(requestedSchema.toAttributes) -}) - -conf.set
spark git commit: [SPARK-7713] [SQL] Use shared broadcast hadoop conf for partitioned table scan.
Repository: spark Updated Branches: refs/heads/branch-1.4 606ae3e10 - 55bd1bb52 [SPARK-7713] [SQL] Use shared broadcast hadoop conf for partitioned table scan. https://issues.apache.org/jira/browse/SPARK-7713 I tested the performance with the following code: ```scala import sqlContext._ import sqlContext.implicits._ (1 to 5000).foreach { i = val df = (1 to 1000).map(j = (j, sstr$j)).toDF(a, b).save(s/tmp/partitioned/i=$i) } sqlContext.sql( CREATE TEMPORARY TABLE partitionedParquet USING org.apache.spark.sql.parquet OPTIONS ( path '/tmp/partitioned' )) table(partitionedParquet).explain(true) ``` In our master `explain` takes 40s in my laptop. With this PR, `explain` takes 14s. Author: Yin Huai yh...@databricks.com Closes #6252 from yhuai/broadcastHadoopConf and squashes the following commits: 6fa73df [Yin Huai] Address comments of Josh and Andrew. 807fbf9 [Yin Huai] Make the new buildScan and SqlNewHadoopRDD private sql. e393555 [Yin Huai] Cheng's comments. 2eb53bb [Yin Huai] Use a shared broadcast Hadoop Configuration for partitioned HadoopFsRelations. (cherry picked from commit b631bf73b9f288f37c98b806be430b22485880e5) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/55bd1bb5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/55bd1bb5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/55bd1bb5 Branch: refs/heads/branch-1.4 Commit: 55bd1bb52e54f710264e6517bb42b74672dd71fb Parents: 606ae3e Author: Yin Huai yh...@databricks.com Authored: Wed May 20 11:23:40 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 20 11:23:49 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 113 +--- .../spark/sql/sources/DataSourceStrategy.scala | 19 +- .../spark/sql/sources/SqlNewHadoopRDD.scala | 268 +++ .../apache/spark/sql/sources/interfaces.scala | 35 ++- 4 files changed, 387 insertions(+), 48 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/55bd1bb5/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 7ca44f7..c35b7ef 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -23,6 +23,7 @@ import scala.collection.JavaConversions._ import scala.util.Try import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -32,13 +33,14 @@ import parquet.hadoop._ import parquet.hadoop.metadata.CompressionCodecName import parquet.hadoop.util.ContextUtil +import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD._ -import org.apache.spark.rdd.{NewHadoopPartition, NewHadoopRDD, RDD} +import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.{Row, SQLConf, SQLContext} -import org.apache.spark.{Logging, Partition = SparkPartition, SparkException} +import org.apache.spark.{Partition = SparkPartition, SparkEnv, SerializableWritable, Logging, SparkException} private[sql] class DefaultSource extends HadoopFsRelationProvider { override def createRelation( @@ -233,40 +235,20 @@ private[sql] class ParquetRelation2( override def buildScan( requiredColumns: Array[String], filters: Array[Filter], - inputFiles: Array[FileStatus]): RDD[Row] = { - -val job = new Job(SparkHadoopUtil.get.conf) -val conf = ContextUtil.getConfiguration(job) - -ParquetInputFormat.setReadSupportClass(job, classOf[RowReadSupport]) - -if (inputFiles.nonEmpty) { - FileInputFormat.setInputPaths(job, inputFiles.map(_.getPath): _*) -} - -// Try to push down filters when filter push-down is enabled. -if (sqlContext.conf.parquetFilterPushDown) { - filters -// Collects all converted Parquet filter predicates. Notice that not all predicates can be -// converted (`ParquetFilters.createFilter` returns an `Option`). That's why a `flatMap` -// is used here. -.flatMap(ParquetFilters.createFilter(dataSchema, _)) -.reduceOption(FilterApi.and) -.foreach(ParquetInputFormat.setFilterPredicate(conf, _)) -} - -conf.set(RowReadSupport.SPARK_ROW_REQUESTED_SCHEMA, { - val requestedSchema = StructType(requiredColumns.map(dataSchema
spark git commit: [SPARK-7320] [SQL] Add Cube / Rollup for dataframe
Repository: spark Updated Branches: refs/heads/branch-1.4 b6182ce89 - 4fd674336 [SPARK-7320] [SQL] Add Cube / Rollup for dataframe This is a follow up for #6257, which broke the maven test. Add cube rollup for DataFrame For example: ```scala testData.rollup($a + $b, $b).agg(sum($a - $b)) testData.cube($a + $b, $b).agg(sum($a - $b)) ``` Author: Cheng Hao hao.ch...@intel.com Closes #6304 from chenghao-intel/rollup and squashes the following commits: 04bb1de [Cheng Hao] move the table register/unregister into beforeAll/afterAll a6069f1 [Cheng Hao] cancel the implicit keyword ced4b8f [Cheng Hao] remove the unnecessary code changes 9959dfa [Cheng Hao] update the code as comments e1d88aa [Cheng Hao] update the code as suggested 03bc3d9 [Cheng Hao] Remove the CubedData RollupedData 5fd62d0 [Cheng Hao] hiden the CubedData RollupedData 5ffb196 [Cheng Hao] Add Cube / Rollup for dataframe (cherry picked from commit 42c592adb381ff20832cce55e0849ed68dd7eee4) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4fd67433 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4fd67433 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4fd67433 Branch: refs/heads/branch-1.4 Commit: 4fd674336cfc13bc0abdd378769d53e337d6aed5 Parents: b6182ce Author: Cheng Hao hao.ch...@intel.com Authored: Wed May 20 19:58:22 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 20 19:58:33 2015 -0700 -- .../scala/org/apache/spark/sql/DataFrame.scala | 104 ++- .../org/apache/spark/sql/GroupedData.scala | 92 +++- .../sql/hive/HiveDataFrameAnalyticsSuite.scala | 69 3 files changed, 237 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4fd67433/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index adad858..d78b4c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -685,7 +685,53 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def groupBy(cols: Column*): GroupedData = new GroupedData(this, cols.map(_.expr)) + def groupBy(cols: Column*): GroupedData = { +GroupedData(this, cols.map(_.expr), GroupedData.GroupByType) + } + + /** + * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * {{{ + * // Compute the average for all numeric columns rolluped by department and group. + * df.rollup($department, $group).avg() + * + * // Compute the max age and average salary, rolluped by department and gender. + * df.rollup($department, $gender).agg(Map( + * salary - avg, + * age - max + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def rollup(cols: Column*): GroupedData = { +GroupedData(this, cols.map(_.expr), GroupedData.RollupType) + } + + /** + * Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * {{{ + * // Compute the average for all numeric columns cubed by department and group. + * df.cube($department, $group).avg() + * + * // Compute the max age and average salary, cubed by department and gender. + * df.cube($department, $gender).agg(Map( + * salary - avg, + * age - max + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def cube(cols: Column*): GroupedData = GroupedData(this, cols.map(_.expr), GroupedData.CubeType) /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -710,7 +756,61 @@ class DataFrame private[sql]( @scala.annotation.varargs def groupBy(col1: String, cols: String*): GroupedData = { val colNames: Seq[String] = col1 +: cols -new GroupedData(this, colNames.map(colName = resolve(colName))) +GroupedData(this, colNames.map(colName = resolve(colName)), GroupedData.GroupByType) + } + + /** + * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * This is a variant of rollup that can only group by
spark git commit: [SPARK-7320] [SQL] Add Cube / Rollup for dataframe
Repository: spark Updated Branches: refs/heads/master 895baf8f7 - 42c592adb [SPARK-7320] [SQL] Add Cube / Rollup for dataframe This is a follow up for #6257, which broke the maven test. Add cube rollup for DataFrame For example: ```scala testData.rollup($a + $b, $b).agg(sum($a - $b)) testData.cube($a + $b, $b).agg(sum($a - $b)) ``` Author: Cheng Hao hao.ch...@intel.com Closes #6304 from chenghao-intel/rollup and squashes the following commits: 04bb1de [Cheng Hao] move the table register/unregister into beforeAll/afterAll a6069f1 [Cheng Hao] cancel the implicit keyword ced4b8f [Cheng Hao] remove the unnecessary code changes 9959dfa [Cheng Hao] update the code as comments e1d88aa [Cheng Hao] update the code as suggested 03bc3d9 [Cheng Hao] Remove the CubedData RollupedData 5fd62d0 [Cheng Hao] hiden the CubedData RollupedData 5ffb196 [Cheng Hao] Add Cube / Rollup for dataframe Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/42c592ad Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/42c592ad Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/42c592ad Branch: refs/heads/master Commit: 42c592adb381ff20832cce55e0849ed68dd7eee4 Parents: 895baf8 Author: Cheng Hao hao.ch...@intel.com Authored: Wed May 20 19:58:22 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 20 19:58:22 2015 -0700 -- .../scala/org/apache/spark/sql/DataFrame.scala | 104 ++- .../org/apache/spark/sql/GroupedData.scala | 92 +++- .../sql/hive/HiveDataFrameAnalyticsSuite.scala | 69 3 files changed, 237 insertions(+), 28 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/42c592ad/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index adad858..d78b4c2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -685,7 +685,53 @@ class DataFrame private[sql]( * @since 1.3.0 */ @scala.annotation.varargs - def groupBy(cols: Column*): GroupedData = new GroupedData(this, cols.map(_.expr)) + def groupBy(cols: Column*): GroupedData = { +GroupedData(this, cols.map(_.expr), GroupedData.GroupByType) + } + + /** + * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * {{{ + * // Compute the average for all numeric columns rolluped by department and group. + * df.rollup($department, $group).avg() + * + * // Compute the max age and average salary, rolluped by department and gender. + * df.rollup($department, $gender).agg(Map( + * salary - avg, + * age - max + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def rollup(cols: Column*): GroupedData = { +GroupedData(this, cols.map(_.expr), GroupedData.RollupType) + } + + /** + * Create a multi-dimensional cube for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * {{{ + * // Compute the average for all numeric columns cubed by department and group. + * df.cube($department, $group).avg() + * + * // Compute the max age and average salary, cubed by department and gender. + * df.cube($department, $gender).agg(Map( + * salary - avg, + * age - max + * )) + * }}} + * @group dfops + * @since 1.4.0 + */ + @scala.annotation.varargs + def cube(cols: Column*): GroupedData = GroupedData(this, cols.map(_.expr), GroupedData.CubeType) /** * Groups the [[DataFrame]] using the specified columns, so we can run aggregation on them. @@ -710,7 +756,61 @@ class DataFrame private[sql]( @scala.annotation.varargs def groupBy(col1: String, cols: String*): GroupedData = { val colNames: Seq[String] = col1 +: cols -new GroupedData(this, colNames.map(colName = resolve(colName))) +GroupedData(this, colNames.map(colName = resolve(colName)), GroupedData.GroupByType) + } + + /** + * Create a multi-dimensional rollup for the current [[DataFrame]] using the specified columns, + * so we can run aggregation on them. + * See [[GroupedData]] for all the available aggregate functions. + * + * This is a variant of rollup that can only group by existing columns using column names + * (i.e. cannot construct expressions). + * + * {{{ + * // Compute the
spark git commit: [SPARK-8020] [SQL] Spark SQL conf in spark-defaults.conf make metadataHive get constructed too early
Repository: spark Updated Branches: refs/heads/master bcb47ad77 - 7b7f7b6c6 [SPARK-8020] [SQL] Spark SQL conf in spark-defaults.conf make metadataHive get constructed too early https://issues.apache.org/jira/browse/SPARK-8020 Author: Yin Huai yh...@databricks.com Closes #6571 from yhuai/SPARK-8020-1 and squashes the following commits: 0398f5b [Yin Huai] First populate the SQLConf and then construct executionHive and metadataHive. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7b7f7b6c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7b7f7b6c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7b7f7b6c Branch: refs/heads/master Commit: 7b7f7b6c6fd903e2ecfc886d29eaa9df58adcfc3 Parents: bcb47ad Author: Yin Huai yh...@databricks.com Authored: Tue Jun 2 00:16:56 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue Jun 2 00:16:56 2015 -0700 -- .../scala/org/apache/spark/sql/SQLContext.scala | 25 +--- 1 file changed, 22 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7b7f7b6c/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7384b24..91e6385 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -182,9 +182,28 @@ class SQLContext(@transient val sparkContext: SparkContext) conf.dialect } - sparkContext.getConf.getAll.foreach { -case (key, value) if key.startsWith(spark.sql) = setConf(key, value) -case _ = + { +// We extract spark sql settings from SparkContext's conf and put them to +// Spark SQL's conf. +// First, we populate the SQLConf (conf). So, we can make sure that other values using +// those settings in their construction can get the correct settings. +// For example, metadataHive in HiveContext may need both spark.sql.hive.metastore.version +// and spark.sql.hive.metastore.jars to get correctly constructed. +val properties = new Properties +sparkContext.getConf.getAll.foreach { + case (key, value) if key.startsWith(spark.sql) = properties.setProperty(key, value) + case _ = +} +// We directly put those settings to conf to avoid of calling setConf, which may have +// side-effects. For example, in HiveContext, setConf may cause executionHive and metadataHive +// get constructed. If we call setConf directly, the constructed metadataHive may have +// wrong settings, or the construction may fail. +conf.setConf(properties) +// After we have populated SQLConf, we call setConf to populate other confs in the subclass +// (e.g. hiveconf in HiveContext). +properties.foreach { + case (key, value) = setConf(key, value) +} } @transient - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8020] [SQL] Spark SQL conf in spark-defaults.conf make metadataHive get constructed too early
Repository: spark Updated Branches: refs/heads/branch-1.4 9d6475b93 - 4940630f5 [SPARK-8020] [SQL] Spark SQL conf in spark-defaults.conf make metadataHive get constructed too early https://issues.apache.org/jira/browse/SPARK-8020 Author: Yin Huai yh...@databricks.com Closes #6571 from yhuai/SPARK-8020-1 and squashes the following commits: 0398f5b [Yin Huai] First populate the SQLConf and then construct executionHive and metadataHive. (cherry picked from commit 7b7f7b6c6fd903e2ecfc886d29eaa9df58adcfc3) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4940630f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4940630f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4940630f Branch: refs/heads/branch-1.4 Commit: 4940630f56d3e95a01526bf1fdfc88517b8e661b Parents: 9d6475b Author: Yin Huai yh...@databricks.com Authored: Tue Jun 2 00:16:56 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue Jun 2 00:17:09 2015 -0700 -- .../scala/org/apache/spark/sql/SQLContext.scala | 25 +--- 1 file changed, 22 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4940630f/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 7384b24..91e6385 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -182,9 +182,28 @@ class SQLContext(@transient val sparkContext: SparkContext) conf.dialect } - sparkContext.getConf.getAll.foreach { -case (key, value) if key.startsWith(spark.sql) = setConf(key, value) -case _ = + { +// We extract spark sql settings from SparkContext's conf and put them to +// Spark SQL's conf. +// First, we populate the SQLConf (conf). So, we can make sure that other values using +// those settings in their construction can get the correct settings. +// For example, metadataHive in HiveContext may need both spark.sql.hive.metastore.version +// and spark.sql.hive.metastore.jars to get correctly constructed. +val properties = new Properties +sparkContext.getConf.getAll.foreach { + case (key, value) if key.startsWith(spark.sql) = properties.setProperty(key, value) + case _ = +} +// We directly put those settings to conf to avoid of calling setConf, which may have +// side-effects. For example, in HiveContext, setConf may cause executionHive and metadataHive +// get constructed. If we call setConf directly, the constructed metadataHive may have +// wrong settings, or the construction may fail. +conf.setConf(properties) +// After we have populated SQLConf, we call setConf to populate other confs in the subclass +// (e.g. hiveconf in HiveContext). +properties.foreach { + case (key, value) = setConf(key, value) +} } @transient - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8121] [SQL] Fixes InsertIntoHadoopFsRelation job initialization for Hadoop 1.x
Repository: spark Updated Branches: refs/heads/master ed5c2dccd - bbdfc0a40 [SPARK-8121] [SQL] Fixes InsertIntoHadoopFsRelation job initialization for Hadoop 1.x For Hadoop 1.x, `TaskAttemptContext` constructor clones the `Configuration` argument, thus configurations done in `HadoopFsRelation.prepareForWriteJob()` are not populated to *driver* side `TaskAttemptContext` (executor side configurations are properly populated). Currently this should only affect Parquet output committer class configuration. Author: Cheng Lian l...@databricks.com Closes #6669 from liancheng/spark-8121 and squashes the following commits: 73819e8 [Cheng Lian] Minor logging fix fce089c [Cheng Lian] Adds more logging b6f78a6 [Cheng Lian] Fixes compilation error introduced while rebasing 963a1aa [Cheng Lian] Addresses @yhuai's comment c3a0b1a [Cheng Lian] Fixes InsertIntoHadoopFsRelation job initialization Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bbdfc0a4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bbdfc0a4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bbdfc0a4 Branch: refs/heads/master Commit: bbdfc0a40fb39760c122e7b9ce80aa1e340e55ee Parents: ed5c2dc Author: Cheng Lian l...@databricks.com Authored: Mon Jun 8 11:34:18 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 8 11:34:18 2015 -0700 -- .../scala/org/apache/spark/sql/SQLConf.scala| 1 + .../apache/spark/sql/parquet/newParquet.scala | 7 +++ .../org/apache/spark/sql/sources/commands.scala | 18 +-- .../spark/sql/parquet/ParquetIOSuite.scala | 52 +--- 4 files changed, 65 insertions(+), 13 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bbdfc0a4/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index c778889..be786f9 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -76,6 +76,7 @@ private[spark] object SQLConf { // The output committer class used by FSBasedRelation. The specified class needs to be a // subclass of org.apache.hadoop.mapreduce.OutputCommitter. + // NOTE: This property should be set in Hadoop `Configuration` rather than Spark `SQLConf` val OUTPUT_COMMITTER_CLASS = spark.sql.sources.outputCommitterClass // Whether to perform eager analysis when constructing a dataframe. http://git-wip-us.apache.org/repos/asf/spark/blob/bbdfc0a4/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 5dda440..7af4eb1 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -212,6 +212,13 @@ private[sql] class ParquetRelation2( classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) +if (conf.get(spark.sql.parquet.output.committer.class) == null) { + logInfo(Using default output committer for Parquet: + +classOf[ParquetOutputCommitter].getCanonicalName) +} else { + logInfo(Using user defined output committer for Parquet: + committerClass.getCanonicalName) +} + conf.setClass( SQLConf.OUTPUT_COMMITTER_CLASS, committerClass, http://git-wip-us.apache.org/repos/asf/spark/blob/bbdfc0a4/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index bd3aad6..c94199b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -297,12 +297,16 @@ private[sql] abstract class BaseWriterContainer( def driverSideSetup(): Unit = { setupIDs(0, 0, 0) setupConf() -taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) -// This preparation must happen before initializing output format and output committer, since -// their initialization involves the job configuration, which can be potentially decorated in -// `relation.prepareJobForWrite`. +// Order of the following two lines is important. For Hadoop 1, TaskAttemptContext constructor +//
spark git commit: [SPARK-8121] [SQL] Fixes InsertIntoHadoopFsRelation job initialization for Hadoop 1.x (branch 1.4 backport based on https://github.com/apache/spark/pull/6669)
Repository: spark Updated Branches: refs/heads/branch-1.4 a3afc2cba - 69197c3e3 [SPARK-8121] [SQL] Fixes InsertIntoHadoopFsRelation job initialization for Hadoop 1.x (branch 1.4 backport based on https://github.com/apache/spark/pull/6669) Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/69197c3e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/69197c3e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/69197c3e Branch: refs/heads/branch-1.4 Commit: 69197c3e382abd477e6eeb87ffbda69bfa68fa14 Parents: a3afc2c Author: Cheng Lian l...@databricks.com Authored: Mon Jun 8 11:35:30 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 8 11:36:42 2015 -0700 -- .../scala/org/apache/spark/sql/SQLConf.scala| 1 + .../apache/spark/sql/parquet/newParquet.scala | 7 +++ .../org/apache/spark/sql/sources/commands.scala | 18 +-- .../spark/sql/parquet/ParquetIOSuite.scala | 50 +--- 4 files changed, 64 insertions(+), 12 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/69197c3e/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala index 77c6af2..26b4e5f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLConf.scala @@ -73,6 +73,7 @@ private[spark] object SQLConf { // The output committer class used by FSBasedRelation. The specified class needs to be a // subclass of org.apache.hadoop.mapreduce.OutputCommitter. + // NOTE: This property should be set in Hadoop `Configuration` rather than Spark `SQLConf` val OUTPUT_COMMITTER_CLASS = spark.sql.sources.outputCommitterClass // Whether to perform eager analysis when constructing a dataframe. http://git-wip-us.apache.org/repos/asf/spark/blob/69197c3e/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index bf55e23..3328e6f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -212,6 +212,13 @@ private[sql] class ParquetRelation2( classOf[ParquetOutputCommitter], classOf[ParquetOutputCommitter]) +if (conf.get(spark.sql.parquet.output.committer.class) == null) { + logInfo(Using default output committer for Parquet: + +classOf[ParquetOutputCommitter].getCanonicalName) +} else { + logInfo(Using user defined output committer for Parquet: + committerClass.getCanonicalName) +} + conf.setClass( SQLConf.OUTPUT_COMMITTER_CLASS, committerClass, http://git-wip-us.apache.org/repos/asf/spark/blob/69197c3e/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 9357a56..eb4e8f8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -297,12 +297,16 @@ private[sql] abstract class BaseWriterContainer( def driverSideSetup(): Unit = { setupIDs(0, 0, 0) setupConf() -taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) -// This preparation must happen before initializing output format and output committer, since -// their initialization involves the job configuration, which can be potentially decorated in -// `relation.prepareJobForWrite`. +// Order of the following two lines is important. For Hadoop 1, TaskAttemptContext constructor +// clones the Configuration object passed in. If we initialize the TaskAttemptContext first, +// configurations made in prepareJobForWrite(job) are not populated into the TaskAttemptContext. +// +// Also, the `prepareJobForWrite` call must happen before initializing output format and output +// committer, since their initialization involve the job configuration, which can be potentially +// decorated in `prepareJobForWrite`. outputWriterFactory = relation.prepareJobForWrite(job) +taskAttemptContext = newTaskAttemptContext(serializableConf.value, taskAttemptId) outputFormatClass = job.getOutputFormatClass
spark git commit: [SPARK-7747] [SQL] [DOCS] spark.sql.planner.externalSort
Repository: spark Updated Branches: refs/heads/master 4f16d3fe2 - 4060526cd [SPARK-7747] [SQL] [DOCS] spark.sql.planner.externalSort Add documentation for spark.sql.planner.externalSort Author: Luca Martinetti l...@luca.io Closes #6272 from lucamartinetti/docs-externalsort and squashes the following commits: 985661b [Luca Martinetti] [SPARK-7747] [SQL] [DOCS] Add documentation for spark.sql.planner.externalSort Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/4060526c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/4060526c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/4060526c Branch: refs/heads/master Commit: 4060526cd3b7e9ba345ce94f6e081cc1156e53ab Parents: 4f16d3f Author: Luca Martinetti l...@luca.io Authored: Fri Jun 5 13:40:11 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 5 13:40:11 2015 -0700 -- docs/sql-programming-guide.md | 7 +++ 1 file changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/4060526c/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 282ea75..cde5830 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1785,6 +1785,13 @@ that these options will be deprecated in future release as more optimizations ar Configures the number of partitions to use when shuffling data for joins or aggregations. /td /tr + tr +tdcodespark.sql.planner.externalSort/code/td +tdfalse/td +td + When true, performs sorts spilling to disk as needed otherwise sort each partition in memory. +/td + /tr /table # Distributed SQL Engine - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7747] [SQL] [DOCS] spark.sql.planner.externalSort
Repository: spark Updated Branches: refs/heads/branch-1.4 200c980a1 - 94f65bcce [SPARK-7747] [SQL] [DOCS] spark.sql.planner.externalSort Add documentation for spark.sql.planner.externalSort Author: Luca Martinetti l...@luca.io Closes #6272 from lucamartinetti/docs-externalsort and squashes the following commits: 985661b [Luca Martinetti] [SPARK-7747] [SQL] [DOCS] Add documentation for spark.sql.planner.externalSort (cherry picked from commit 4060526cd3b7e9ba345ce94f6e081cc1156e53ab) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/94f65bcc Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/94f65bcc Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/94f65bcc Branch: refs/heads/branch-1.4 Commit: 94f65bcceebd2850e0dc446a1b56bb01d54d19d7 Parents: 200c980 Author: Luca Martinetti l...@luca.io Authored: Fri Jun 5 13:40:11 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 5 13:41:52 2015 -0700 -- docs/sql-programming-guide.md | 7 +++ 1 file changed, 7 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/94f65bcc/docs/sql-programming-guide.md -- diff --git a/docs/sql-programming-guide.md b/docs/sql-programming-guide.md index 282ea75..cde5830 100644 --- a/docs/sql-programming-guide.md +++ b/docs/sql-programming-guide.md @@ -1785,6 +1785,13 @@ that these options will be deprecated in future release as more optimizations ar Configures the number of partitions to use when shuffling data for joins or aggregations. /td /tr + tr +tdcodespark.sql.planner.externalSort/code/td +tdfalse/td +td + When true, performs sorts spilling to disk as needed otherwise sort each partition in memory. +/td + /tr /table # Distributed SQL Engine - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-6964] [SQL] Support Cancellation in the Thrift Server
Repository: spark Updated Branches: refs/heads/master 6ebe419f3 - eb19d3f75 [SPARK-6964] [SQL] Support Cancellation in the Thrift Server Support runInBackground in SparkExecuteStatementOperation, and add cancellation Author: Dong Wang d...@databricks.com Closes #6207 from dongwang218/SPARK-6964-jdbc-cancel and squashes the following commits: 687c113 [Dong Wang] fix 100 characters 7bfa2a7 [Dong Wang] fix merge 380480f [Dong Wang] fix for liancheng's comments eb3e385 [Dong Wang] small nit 341885b [Dong Wang] small fix 3d8ebf8 [Dong Wang] add spark.sql.hive.thriftServer.async flag 04142c3 [Dong Wang] set SQLSession for async execution 184ec35 [Dong Wang] keep hive conf 819ae03 [Dong Wang] [SPARK-6964][SQL][WIP] Support Cancellation in the Thrift Server Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/eb19d3f7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/eb19d3f7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/eb19d3f7 Branch: refs/heads/master Commit: eb19d3f75cbd002f7e72ce02017a8de67f562792 Parents: 6ebe419 Author: Dong Wang d...@databricks.com Authored: Fri Jun 5 17:41:12 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 5 17:41:12 2015 -0700 -- .../scala/org/apache/spark/sql/SQLContext.scala | 5 + .../SparkExecuteStatementOperation.scala| 164 +-- .../server/SparkSQLOperationManager.scala | 7 +- .../thriftserver/HiveThriftServer2Suites.scala | 42 - .../org/apache/spark/sql/hive/HiveContext.scala | 6 + 5 files changed, 208 insertions(+), 16 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/eb19d3f7/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index 0aab7fa..ddb5402 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -916,6 +916,11 @@ class SQLContext(@transient val sparkContext: SparkContext) tlSession.remove() } + protected[sql] def setSession(session: SQLSession): Unit = { +detachSession() +tlSession.set(session) + } + protected[sql] class SQLSession { // Note that this is a lazy val so we can override the default value in subclasses. protected[sql] lazy val conf: SQLConf = new SQLConf http://git-wip-us.apache.org/repos/asf/spark/blob/eb19d3f7/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala index c0d1266..e071103 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkExecuteStatementOperation.scala @@ -17,11 +17,23 @@ package org.apache.spark.sql.hive.thriftserver +import java.security.PrivilegedExceptionAction import java.sql.{Date, Timestamp} +import java.util.concurrent.RejectedExecutionException import java.util.{Map = JMap, UUID} +import scala.collection.JavaConversions._ +import scala.collection.mutable.{ArrayBuffer, Map = SMap} +import scala.util.control.NonFatal + +import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.metastore.api.FieldSchema import org.apache.hive.service.cli._ +import org.apache.hadoop.hive.ql.metadata.Hive +import org.apache.hadoop.hive.ql.metadata.HiveException +import org.apache.hadoop.hive.ql.session.SessionState +import org.apache.hadoop.hive.shims.ShimLoader +import org.apache.hadoop.security.UserGroupInformation import org.apache.hive.service.cli.operation.ExecuteStatementOperation import org.apache.hive.service.cli.session.HiveSession @@ -31,8 +43,6 @@ import org.apache.spark.sql.hive.{HiveContext, HiveMetastoreTypes} import org.apache.spark.sql.types._ import org.apache.spark.sql.{DataFrame, Row = SparkRow, SQLConf} -import scala.collection.JavaConversions._ -import scala.collection.mutable.{ArrayBuffer, Map = SMap} private[hive] class SparkExecuteStatementOperation( parentSession: HiveSession, @@ -40,17 +50,19 @@ private[hive] class SparkExecuteStatementOperation( confOverlay: JMap[String, String], runInBackground: Boolean = true) (hiveContext: HiveContext, sessionToActivePool: SMap[SessionHandle, String])
spark git commit: [SPARK-8014] [SQL] Avoid premature metadata discovery when writing a HadoopFsRelation with a save mode other than Append
Repository: spark Updated Branches: refs/heads/branch-1.4 815e05654 - cbaf59544 [SPARK-8014] [SQL] Avoid premature metadata discovery when writing a HadoopFsRelation with a save mode other than Append The current code references the schema of the DataFrame to be written before checking save mode. This triggers expensive metadata discovery prematurely. For save mode other than `Append`, this metadata discovery is useless since we either ignore the result (for `Ignore` and `ErrorIfExists`) or delete existing files (for `Overwrite`) later. This PR fixes this issue by deferring metadata discovery after save mode checking. Author: Cheng Lian l...@databricks.com Closes #6583 from liancheng/spark-8014 and squashes the following commits: 1aafabd [Cheng Lian] Updates comments 088abaa [Cheng Lian] Avoids schema merging and partition discovery when data schema and partition schema are defined 8fbd93f [Cheng Lian] Fixes SPARK-8014 (cherry picked from commit 686a45f0b9c50ede2a80854ed6a155ee8a9a4f5c) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/cbaf5954 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/cbaf5954 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/cbaf5954 Branch: refs/heads/branch-1.4 Commit: cbaf595447ae42227516f9220f6a0ed2d9fec54f Parents: 815e056 Author: Cheng Lian l...@databricks.com Authored: Tue Jun 2 13:32:13 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue Jun 2 13:32:34 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 2 +- .../org/apache/spark/sql/sources/commands.scala | 20 +-- .../org/apache/spark/sql/sources/ddl.scala | 16 ++ .../apache/spark/sql/sources/interfaces.scala | 2 +- .../sql/sources/hadoopFsRelationSuites.scala| 59 +++- 5 files changed, 67 insertions(+), 32 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/cbaf5954/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index e439a18..824ae36 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -190,7 +190,7 @@ private[sql] class ParquetRelation2( } } - override def dataSchema: StructType = metadataCache.dataSchema + override def dataSchema: StructType = maybeDataSchema.getOrElse(metadataCache.dataSchema) override private[sql] def refresh(): Unit = { super.refresh() http://git-wip-us.apache.org/repos/asf/spark/blob/cbaf5954/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 3132067..71f016b 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -30,9 +30,10 @@ import org.apache.spark._ import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil import org.apache.spark.sql.catalyst.CatalystTypeConverters +import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateProjection -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan +import org.apache.spark.sql.catalyst.plans.logical.{Project, LogicalPlan} import org.apache.spark.sql.execution.RunnableCommand import org.apache.spark.sql.types.StructType import org.apache.spark.sql.{DataFrame, SQLConf, SQLContext, SaveMode} @@ -94,10 +95,19 @@ private[sql] case class InsertIntoHadoopFsRelation( // We create a DataFrame by applying the schema of relation to the data to make sure. // We are writing data based on the expected schema, - val df = sqlContext.createDataFrame( -DataFrame(sqlContext, query).queryExecution.toRdd, -relation.schema, -needsConversion = false) + val df = { +// For partitioned relation r, r.schema's column ordering can be different from the column +// ordering of data.logicalPlan (partition columns are all moved after data column). We +// need a Project to adjust the ordering, so that inside InsertIntoHadoopFsRelation, we can +// safely apply the schema of r.schema to the data. +val project = Project( +
spark git commit: [SPARK-7973] [SQL] Increase the timeout of two CliSuite tests.
Repository: spark Updated Branches: refs/heads/branch-1.4 ee7f365bd - 54a4ea407 [SPARK-7973] [SQL] Increase the timeout of two CliSuite tests. https://issues.apache.org/jira/browse/SPARK-7973 Author: Yin Huai yh...@databricks.com Closes #6525 from yhuai/SPARK-7973 and squashes the following commits: 763b821 [Yin Huai] Also change the timeout of Single command with -e to 2 minutes. e598a08 [Yin Huai] Increase the timeout to 3 minutes. (cherry picked from commit f1646e1023bd03e27268a8aa2ea11b6cc284075f) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/54a4ea40 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/54a4ea40 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/54a4ea40 Branch: refs/heads/branch-1.4 Commit: 54a4ea4078b2a423f2b20e0dbd2290004e251e0d Parents: ee7f365 Author: Yin Huai yh...@databricks.com Authored: Wed Jun 3 09:26:21 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed Jun 3 09:26:30 2015 -0700 -- .../scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/54a4ea40/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala -- diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index cc07db8..eb3a315 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -133,7 +133,7 @@ class CliSuite extends FunSuite with BeforeAndAfter with Logging { } test(Single command with -e) { -runCliWithin(1.minute, Seq(-e, SHOW DATABASES;))( - OK) +runCliWithin(2.minute, Seq(-e, SHOW DATABASES;))( - OK) } test(Single command with --database) { @@ -165,7 +165,7 @@ class CliSuite extends FunSuite with BeforeAndAfter with Logging { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource(data/files/small_kv.txt) -runCliWithin(1.minute, Seq(--jars, s$jarFile))( +runCliWithin(3.minute, Seq(--jars, s$jarFile))( CREATE TABLE t1(key string, val string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; .stripMargin - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7973] [SQL] Increase the timeout of two CliSuite tests.
Repository: spark Updated Branches: refs/heads/master 28dbde387 - f1646e102 [SPARK-7973] [SQL] Increase the timeout of two CliSuite tests. https://issues.apache.org/jira/browse/SPARK-7973 Author: Yin Huai yh...@databricks.com Closes #6525 from yhuai/SPARK-7973 and squashes the following commits: 763b821 [Yin Huai] Also change the timeout of Single command with -e to 2 minutes. e598a08 [Yin Huai] Increase the timeout to 3 minutes. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f1646e10 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f1646e10 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f1646e10 Branch: refs/heads/master Commit: f1646e1023bd03e27268a8aa2ea11b6cc284075f Parents: 28dbde3 Author: Yin Huai yh...@databricks.com Authored: Wed Jun 3 09:26:21 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed Jun 3 09:26:21 2015 -0700 -- .../scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f1646e10/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala -- diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 3732af7..13b0c59 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -133,7 +133,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { } test(Single command with -e) { -runCliWithin(1.minute, Seq(-e, SHOW DATABASES;))( - OK) +runCliWithin(2.minute, Seq(-e, SHOW DATABASES;))( - OK) } test(Single command with --database) { @@ -165,7 +165,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { val dataFilePath = Thread.currentThread().getContextClassLoader.getResource(data/files/small_kv.txt) -runCliWithin(1.minute, Seq(--jars, s$jarFile))( +runCliWithin(3.minute, Seq(--jars, s$jarFile))( CREATE TABLE t1(key string, val string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; .stripMargin - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8406] [SQL] Backports SPARK-8406 and PR #6864 to branch-1.4
Repository: spark Updated Branches: refs/heads/branch-1.4 b836bac3f - 451c8722a [SPARK-8406] [SQL] Backports SPARK-8406 and PR #6864 to branch-1.4 Author: Cheng Lian l...@databricks.com Closes #6932 from liancheng/spark-8406-for-1.4 and squashes the following commits: a0168fe [Cheng Lian] Backports SPARK-8406 and PR #6864 to branch-1.4 Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/451c8722 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/451c8722 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/451c8722 Branch: refs/heads/branch-1.4 Commit: 451c8722afea83e8e8f11c438469eea10e5acf4c Parents: b836bac Author: Cheng Lian l...@databricks.com Authored: Mon Jun 22 10:04:29 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 22 10:04:29 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 41 ++ .../org/apache/spark/sql/sources/commands.scala | 59 .../spark/sql/hive/orc/OrcFileOperator.scala| 7 ++- .../apache/spark/sql/hive/orc/OrcRelation.scala | 3 +- .../apache/spark/sql/hive/test/TestHive.scala | 2 +- .../spark/sql/hive/orc/OrcSourceSuite.scala | 22 .../spark/sql/sources/SimpleTextRelation.scala | 4 +- .../sql/sources/hadoopFsRelationSuites.scala| 39 +++-- 8 files changed, 110 insertions(+), 67 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/451c8722/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 3328e6f..abf9614 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -61,50 +61,21 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext extends OutputWriter { private val recordWriter: RecordWriter[Void, Row] = { -val conf = context.getConfiguration val outputFormat = { - // When appending new Parquet files to an existing Parquet file directory, to avoid - // overwriting existing data files, we need to find out the max task ID encoded in these data - // file names. - // TODO Make this snippet a utility function for other data source developers - val maxExistingTaskId = { -// Note that `path` may point to a temporary location. Here we retrieve the real -// destination path from the configuration -val outputPath = new Path(conf.get(spark.sql.sources.output.path)) -val fs = outputPath.getFileSystem(conf) - -if (fs.exists(outputPath)) { - // Pattern used to match task ID in part file names, e.g.: - // - // part-r-1.gz.parquet - // ^ - val partFilePattern = part-.-(\d{1,}).*.r - - fs.listStatus(outputPath).map(_.getPath.getName).map { -case partFilePattern(id) = id.toInt -case name if name.startsWith(_) = 0 -case name if name.startsWith(.) = 0 -case name = throw new AnalysisException( - sTrying to write Parquet files to directory $outputPath, + -sbut found items with illegal name '$name'.) - }.reduceOption(_ max _).getOrElse(0) -} else { - 0 -} - } - new ParquetOutputFormat[Row]() { // Here we override `getDefaultWorkFile` for two reasons: // -// 1. To allow appending. We need to generate output file name based on the max available -// task ID computed above. +// 1. To allow appending. We need to generate unique output file names to avoid +// overwriting existing files (either exist before the write job, or are just written +// by other tasks within the same write job). // // 2. To allow dynamic partitioning. Default `getDefaultWorkFile` uses // `FileOutputCommitter.getWorkPath()`, which points to the base directory of all // partitions in the case of dynamic partitioning. override def getDefaultWorkFile(context: TaskAttemptContext, extension: String): Path = { - val split = context.getTaskAttemptID.getTaskID.getId + maxExistingTaskId + 1 - new Path(path, fpart-r-$split%05d$extension) + val uniqueWriteJobId = context.getConfiguration.get(spark.sql.sources.writeJobUUID) + val split = context.getTaskAttemptID.getTaskID.getId + new Path(path, fpart-r-$split%05d-$uniqueWriteJobId$extension) } } }
spark git commit: [SPARK-8406] [SQL] Adding UUID to output file name to avoid accidental overwriting
Repository: spark Updated Branches: refs/heads/master 47c1d5629 - 0818fdec3 [SPARK-8406] [SQL] Adding UUID to output file name to avoid accidental overwriting This PR fixes a Parquet output file name collision bug which may cause data loss. Changes made: 1. Identify each write job issued by `InsertIntoHadoopFsRelation` with a UUID All concrete data sources which extend `HadoopFsRelation` (Parquet and ORC for now) must use this UUID to generate task output file path to avoid name collision. 2. Make `TestHive` use a local mode `SparkContext` with 32 threads to increase parallelism The major reason for this is that, the original parallelism of 2 is too low to reproduce the data loss issue. Also, higher concurrency may potentially caught more concurrency bugs during testing phase. (It did help us spotted SPARK-8501.) 3. `OrcSourceSuite` was updated to workaround SPARK-8501, which we detected along the way. NOTE: This PR is made a little bit more complicated than expected because we hit two other bugs on the way and have to work them around. See [SPARK-8501] [1] and [SPARK-8513] [2]. [1]: https://github.com/liancheng/spark/tree/spark-8501 [2]: https://github.com/liancheng/spark/tree/spark-8513 Some background and a summary of offline discussion with yhuai about this issue for better understanding: In 1.4.0, we added `HadoopFsRelation` to abstract partition support of all data sources that are based on Hadoop `FileSystem` interface. Specifically, this makes partition discovery, partition pruning, and writing dynamic partitions for data sources much easier. To support appending, the Parquet data source tries to find out the max part number of part-files in the destination directory (i.e., `id` in output file name `part-r-id.gz.parquet`) at the beginning of the write job. In 1.3.0, this step happens on driver side before any files are written. However, in 1.4.0, this is moved to task side. Unfortunately, for tasks scheduled later, they may see wrong max part number generated of files newly written by other finished tasks within the same job. This actually causes a race condition. In most cases, this only causes nonconsecutive part numbers in output file names. But when the DataFrame contains thousands of RDD partitions, it's likely that two tasks may choose the same part number, then one of them gets overwritten by the other. Before `HadoopFsRelation`, Spark SQL already supports appending data to Hive tables. From a user's perspective, these two look similar. However, they differ a lot internally. When data are inserted into Hive tables via Spark SQL, `InsertIntoHiveTable` simulates Hive's behaviors: 1. Write data to a temporary location 2. Move data in the temporary location to the final destination location using - `Hive.loadTable()` for non-partitioned table - `Hive.loadPartition()` for static partitions - `Hive.loadDynamicPartitions()` for dynamic partitions The important part is that, `Hive.copyFiles()` is invoked in step 2 to move the data to the destination directory (I found the name is kinda confusing since no copying occurs here, we are just moving and renaming stuff). If a file in the source directory and another file in the destination directory happen to have the same name, say `part-r-1.parquet`, the former is moved to the destination directory and renamed with a `_copy_N` postfix (`part-r-1_copy_1.parquet`). That's how Hive handles appending and avoids name collision between different write jobs. Some alternatives fixes considered for this issue: 1. Use a similar approach as Hive This approach is not preferred in Spark 1.4.0 mainly because file metadata operations in S3 tend to be slow, especially for tables with lots of file and/or partitions. That's why `InsertIntoHadoopFsRelation` just inserts to destination directory directly, and is often used together with `DirectParquetOutputCommitter` to reduce latency when working with S3. This means, we don't have the chance to do renaming, and must avoid name collision from the very beginning. 2. Same as 1.3, just move max part number detection back to driver side This isn't doable because unlike 1.3, 1.4 also takes dynamic partitioning into account. When inserting into dynamic partitions, we don't know which partition directories will be touched on driver side before issuing the write job. Checking all partition directories is simply too expensive for tables with thousands of partitions. 3. Add extra component to output file names to avoid name collision This seems to be the only reasonable solution for now. To be more specific, we need a JOB level unique identifier to identify all write jobs issued by `InsertIntoHadoopFile`. Notice that TASK level unique identifiers can NOT be used. Because in this way a speculative task will write to a different output file from the original task. If both tasks
spark git commit: [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings (branch-1.4)
Repository: spark Updated Branches: refs/heads/branch-1.4 451c8722a - 65981619b [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings (branch-1.4) This is branch 1.4 backport of https://github.com/apache/spark/pull/6888. Below is the original description. In earlier versions of Spark SQL we casted `TimestampType` and `DataType` to `StringType` when it was involved in a binary comparison with a `StringType`. This allowed comparing a timestamp with a partial date as a user would expect. - `time 2014-06-10` - `time 2014` In 1.4.0 we tried to cast the String instead into a Timestamp. However, since partial dates are not a valid complete timestamp this results in `null` which results in the tuple being filtered. This PR restores the earlier behavior. Note that we still special case equality so that these comparisons are not affected by not printing zeros for subsecond precision. Author: Michael Armbrust michaeldatabricks.com Closes #6888 from marmbrus/timeCompareString and squashes the following commits: bdef29c [Michael Armbrust] test partial date 1f09adf [Michael Armbrust] special handling of equality 1172c60 [Michael Armbrust] more test fixing 4dfc412 [Michael Armbrust] fix tests aaa9508 [Michael Armbrust] newline 04d908f [Michael Armbrust] [SPARK-8420][SQL] Fix comparision of timestamps/dates with strings Conflicts: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/HiveTypeCoercion.scala sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/predicates.scala Author: Michael Armbrust mich...@databricks.com Closes #6914 from yhuai/timeCompareString-1.4 and squashes the following commits: 9882915 [Michael Armbrust] [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/65981619 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/65981619 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/65981619 Branch: refs/heads/branch-1.4 Commit: 65981619b26da03f0c5133133e318a180235e96d Parents: 451c872 Author: Michael Armbrust mich...@databricks.com Authored: Mon Jun 22 10:45:33 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 22 10:45:33 2015 -0700 -- .../catalyst/analysis/HiveTypeCoercion.scala| 17 -- .../sql/catalyst/expressions/predicates.scala | 9 .../apache/spark/sql/DataFrameDateSuite.scala | 56 .../org/apache/spark/sql/SQLQuerySuite.scala| 4 ++ .../scala/org/apache/spark/sql/TestData.scala | 6 --- .../columnar/InMemoryColumnarQuerySuite.scala | 7 ++- 6 files changed, 88 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/65981619/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 fa7968e..6d0f4a0 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 @@ -242,7 +242,16 @@ trait HiveTypeCoercion { case a: BinaryArithmetic if a.right.dataType == StringType = a.makeCopy(Array(a.left, Cast(a.right, DoubleType))) - // we should cast all timestamp/date/string compare into string compare + // For equality between string and timestamp we cast the string to a timestamp + // so that things like rounding of subsecond precision does not affect the comparison. + case p @ Equality(left @ StringType(), right @ TimestampType()) = +p.makeCopy(Array(Cast(left, TimestampType), right)) + case p @ Equality(left @ TimestampType(), right @ StringType()) = +p.makeCopy(Array(left, Cast(right, TimestampType))) + + // We should cast all relative timestamp/date/string comparison into string comparisions + // This behaves as a user would expect because timestamp strings sort lexicographically. + // i.e. TimeStamp(2013-01-01 00:00 ...) 2014 = true case p: BinaryComparison if p.left.dataType == StringType p.right.dataType == DateType = p.makeCopy(Array(p.left, Cast(p.right, StringType))) @@ -251,10 +260,12 @@ trait HiveTypeCoercion { p.makeCopy(Array(Cast(p.left, StringType), p.right)) case p: BinaryComparison if p.left.dataType == StringType p.right.dataType == TimestampType = -p.makeCopy(Array(Cast(p.left, TimestampType
spark git commit: [SPARK-8532] [SQL] In Python's DataFrameWriter, save/saveAsTable/json/parquet/jdbc always override mode
Repository: spark Updated Branches: refs/heads/branch-1.4 507381d39 - 994abbaeb [SPARK-8532] [SQL] In Python's DataFrameWriter, save/saveAsTable/json/parquet/jdbc always override mode https://issues.apache.org/jira/browse/SPARK-8532 This PR has two changes. First, it fixes the bug that save actions (i.e. `save/saveAsTable/json/parquet/jdbc`) always override mode. Second, it adds input argument `partitionBy` to `save/saveAsTable/parquet`. Author: Yin Huai yh...@databricks.com Closes #6937 from yhuai/SPARK-8532 and squashes the following commits: f972d5d [Yin Huai] davies's comment. d37abd2 [Yin Huai] style. d21290a [Yin Huai] Python doc. 889eb25 [Yin Huai] Minor refactoring and add partitionBy to save, saveAsTable, and parquet. 7fbc24b [Yin Huai] Use None instead of error as the default value of mode since JVM-side already uses error as the default value. d696dff [Yin Huai] Python style. 88eb6c4 [Yin Huai] If mode is error, do not call mode method. c40c461 [Yin Huai] Regression test. (cherry picked from commit 5ab9fcfb01a0ad2f6c103f67c1a785d3b49e33f0) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/994abbae Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/994abbae Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/994abbae Branch: refs/heads/branch-1.4 Commit: 994abbaeb3c5444d09548291f865373ba4f1909f Parents: 507381d Author: Yin Huai yh...@databricks.com Authored: Mon Jun 22 13:51:23 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 22 13:51:34 2015 -0700 -- python/pyspark/sql/readwriter.py | 30 +++--- python/pyspark/sql/tests.py | 32 2 files changed, 51 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/994abbae/python/pyspark/sql/readwriter.py -- diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index f036644..1b7bc0f 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -218,7 +218,10 @@ class DataFrameWriter(object): df.write.mode('append').parquet(os.path.join(tempfile.mkdtemp(), 'data')) -self._jwrite = self._jwrite.mode(saveMode) +# At the JVM side, the default value of mode is already set to error. +# So, if the given saveMode is None, we will not call JVM-side's mode method. +if saveMode is not None: +self._jwrite = self._jwrite.mode(saveMode) return self @since(1.4) @@ -253,11 +256,12 @@ class DataFrameWriter(object): if len(cols) == 1 and isinstance(cols[0], (list, tuple)): cols = cols[0] -self._jwrite = self._jwrite.partitionBy(_to_seq(self._sqlContext._sc, cols)) +if len(cols) 0: +self._jwrite = self._jwrite.partitionBy(_to_seq(self._sqlContext._sc, cols)) return self @since(1.4) -def save(self, path=None, format=None, mode=error, **options): +def save(self, path=None, format=None, mode=None, partitionBy=(), **options): Saves the contents of the :class:`DataFrame` to a data source. The data source is specified by the ``format`` and a set of ``options``. @@ -272,11 +276,12 @@ class DataFrameWriter(object): * ``overwrite``: Overwrite existing data. * ``ignore``: Silently ignore this operation if data already exists. * ``error`` (default case): Throw an exception if data already exists. +:param partitionBy: names of partitioning columns :param options: all other string options df.write.mode('append').parquet(os.path.join(tempfile.mkdtemp(), 'data')) -self.mode(mode).options(**options) +self.partitionBy(partitionBy).mode(mode).options(**options) if format is not None: self.format(format) if path is None: @@ -296,7 +301,7 @@ class DataFrameWriter(object): self._jwrite.mode(overwrite if overwrite else append).insertInto(tableName) @since(1.4) -def saveAsTable(self, name, format=None, mode=error, **options): +def saveAsTable(self, name, format=None, mode=None, partitionBy=(), **options): Saves the content of the :class:`DataFrame` as the specified table. In the case the table already exists, behavior of this function depends on the @@ -312,15 +317,16 @@ class DataFrameWriter(object): :param name: the table name :param format: the format used to save :param mode: one of `append`, `overwrite`, `error`, `ignore` (default: error) +:param partitionBy: names of partitioning columns
spark git commit: [SPARK-8532] [SQL] In Python's DataFrameWriter, save/saveAsTable/json/parquet/jdbc always override mode
Repository: spark Updated Branches: refs/heads/master da7bbb943 - 5ab9fcfb0 [SPARK-8532] [SQL] In Python's DataFrameWriter, save/saveAsTable/json/parquet/jdbc always override mode https://issues.apache.org/jira/browse/SPARK-8532 This PR has two changes. First, it fixes the bug that save actions (i.e. `save/saveAsTable/json/parquet/jdbc`) always override mode. Second, it adds input argument `partitionBy` to `save/saveAsTable/parquet`. Author: Yin Huai yh...@databricks.com Closes #6937 from yhuai/SPARK-8532 and squashes the following commits: f972d5d [Yin Huai] davies's comment. d37abd2 [Yin Huai] style. d21290a [Yin Huai] Python doc. 889eb25 [Yin Huai] Minor refactoring and add partitionBy to save, saveAsTable, and parquet. 7fbc24b [Yin Huai] Use None instead of error as the default value of mode since JVM-side already uses error as the default value. d696dff [Yin Huai] Python style. 88eb6c4 [Yin Huai] If mode is error, do not call mode method. c40c461 [Yin Huai] Regression test. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5ab9fcfb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5ab9fcfb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5ab9fcfb Branch: refs/heads/master Commit: 5ab9fcfb01a0ad2f6c103f67c1a785d3b49e33f0 Parents: da7bbb9 Author: Yin Huai yh...@databricks.com Authored: Mon Jun 22 13:51:23 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 22 13:51:23 2015 -0700 -- python/pyspark/sql/readwriter.py | 30 +++--- python/pyspark/sql/tests.py | 32 2 files changed, 51 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5ab9fcfb/python/pyspark/sql/readwriter.py -- diff --git a/python/pyspark/sql/readwriter.py b/python/pyspark/sql/readwriter.py index f036644..1b7bc0f 100644 --- a/python/pyspark/sql/readwriter.py +++ b/python/pyspark/sql/readwriter.py @@ -218,7 +218,10 @@ class DataFrameWriter(object): df.write.mode('append').parquet(os.path.join(tempfile.mkdtemp(), 'data')) -self._jwrite = self._jwrite.mode(saveMode) +# At the JVM side, the default value of mode is already set to error. +# So, if the given saveMode is None, we will not call JVM-side's mode method. +if saveMode is not None: +self._jwrite = self._jwrite.mode(saveMode) return self @since(1.4) @@ -253,11 +256,12 @@ class DataFrameWriter(object): if len(cols) == 1 and isinstance(cols[0], (list, tuple)): cols = cols[0] -self._jwrite = self._jwrite.partitionBy(_to_seq(self._sqlContext._sc, cols)) +if len(cols) 0: +self._jwrite = self._jwrite.partitionBy(_to_seq(self._sqlContext._sc, cols)) return self @since(1.4) -def save(self, path=None, format=None, mode=error, **options): +def save(self, path=None, format=None, mode=None, partitionBy=(), **options): Saves the contents of the :class:`DataFrame` to a data source. The data source is specified by the ``format`` and a set of ``options``. @@ -272,11 +276,12 @@ class DataFrameWriter(object): * ``overwrite``: Overwrite existing data. * ``ignore``: Silently ignore this operation if data already exists. * ``error`` (default case): Throw an exception if data already exists. +:param partitionBy: names of partitioning columns :param options: all other string options df.write.mode('append').parquet(os.path.join(tempfile.mkdtemp(), 'data')) -self.mode(mode).options(**options) +self.partitionBy(partitionBy).mode(mode).options(**options) if format is not None: self.format(format) if path is None: @@ -296,7 +301,7 @@ class DataFrameWriter(object): self._jwrite.mode(overwrite if overwrite else append).insertInto(tableName) @since(1.4) -def saveAsTable(self, name, format=None, mode=error, **options): +def saveAsTable(self, name, format=None, mode=None, partitionBy=(), **options): Saves the content of the :class:`DataFrame` as the specified table. In the case the table already exists, behavior of this function depends on the @@ -312,15 +317,16 @@ class DataFrameWriter(object): :param name: the table name :param format: the format used to save :param mode: one of `append`, `overwrite`, `error`, `ignore` (default: error) +:param partitionBy: names of partitioning columns :param options: all other string options -self.mode(mode).options(**options
spark git commit: [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings
Repository: spark Updated Branches: refs/heads/master 9814b971f - a333a72e0 [SPARK-8420] [SQL] Fix comparision of timestamps/dates with strings In earlier versions of Spark SQL we casted `TimestampType` and `DataType` to `StringType` when it was involved in a binary comparison with a `StringType`. This allowed comparing a timestamp with a partial date as a user would expect. - `time 2014-06-10` - `time 2014` In 1.4.0 we tried to cast the String instead into a Timestamp. However, since partial dates are not a valid complete timestamp this results in `null` which results in the tuple being filtered. This PR restores the earlier behavior. Note that we still special case equality so that these comparisons are not affected by not printing zeros for subsecond precision. Author: Michael Armbrust mich...@databricks.com Closes #6888 from marmbrus/timeCompareString and squashes the following commits: bdef29c [Michael Armbrust] test partial date 1f09adf [Michael Armbrust] special handling of equality 1172c60 [Michael Armbrust] more test fixing 4dfc412 [Michael Armbrust] fix tests aaa9508 [Michael Armbrust] newline 04d908f [Michael Armbrust] [SPARK-8420][SQL] Fix comparision of timestamps/dates with strings Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a333a72e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a333a72e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a333a72e Branch: refs/heads/master Commit: a333a72e029d2546a66b36d6b3458e965430c530 Parents: 9814b97 Author: Michael Armbrust mich...@databricks.com Authored: Fri Jun 19 16:54:51 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 19 16:54:51 2015 -0700 -- .../catalyst/analysis/HiveTypeCoercion.scala| 17 -- .../sql/catalyst/expressions/predicates.scala | 9 .../apache/spark/sql/DataFrameDateSuite.scala | 56 .../org/apache/spark/sql/SQLQuerySuite.scala| 4 ++ .../scala/org/apache/spark/sql/TestData.scala | 6 --- .../columnar/InMemoryColumnarQuerySuite.scala | 7 ++- 6 files changed, 88 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a333a72e/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 8012b22..d4ab1fc 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 @@ -277,15 +277,26 @@ trait HiveTypeCoercion { case a @ BinaryArithmetic(left, right @ StringType()) = a.makeCopy(Array(left, Cast(right, DoubleType))) - // we should cast all timestamp/date/string compare into string compare + // For equality between string and timestamp we cast the string to a timestamp + // so that things like rounding of subsecond precision does not affect the comparison. + case p @ Equality(left @ StringType(), right @ TimestampType()) = +p.makeCopy(Array(Cast(left, TimestampType), right)) + case p @ Equality(left @ TimestampType(), right @ StringType()) = +p.makeCopy(Array(left, Cast(right, TimestampType))) + + // We should cast all relative timestamp/date/string comparison into string comparisions + // This behaves as a user would expect because timestamp strings sort lexicographically. + // i.e. TimeStamp(2013-01-01 00:00 ...) 2014 = true case p @ BinaryComparison(left @ StringType(), right @ DateType()) = p.makeCopy(Array(left, Cast(right, StringType))) case p @ BinaryComparison(left @ DateType(), right @ StringType()) = p.makeCopy(Array(Cast(left, StringType), right)) case p @ BinaryComparison(left @ StringType(), right @ TimestampType()) = -p.makeCopy(Array(Cast(left, TimestampType), right)) +p.makeCopy(Array(left, Cast(right, StringType))) case p @ BinaryComparison(left @ TimestampType(), right @ StringType()) = -p.makeCopy(Array(left, Cast(right, TimestampType))) +p.makeCopy(Array(Cast(left, StringType), right)) + + // Comparisons between dates and timestamps. case p @ BinaryComparison(left @ TimestampType(), right @ DateType()) = p.makeCopy(Array(Cast(left, StringType), Cast(right, StringType))) case p @ BinaryComparison(left @ DateType(), right @ TimestampType()) =
spark git commit: [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents
Repository: spark Updated Branches: refs/heads/branch-1.4 1a6b51078 - 0131142d9 [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents Author: Nathan Howell nhow...@godaddy.com Closes #6799 from NathanHowell/spark-8093 and squashes the following commits: 76ac3e8 [Nathan Howell] [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents (cherry picked from commit 9814b971f07dff8a99f1b8ad2adf70614f1c690b) Signed-off-by: Yin Huai yh...@databricks.com Conflicts: sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0131142d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0131142d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0131142d Branch: refs/heads/branch-1.4 Commit: 0131142d98b191f6cc112d383aa10582a3ac35bf Parents: 1a6b510 Author: Nathan Howell nhow...@godaddy.com Authored: Fri Jun 19 16:19:28 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 19 16:23:11 2015 -0700 -- .../org/apache/spark/sql/json/InferSchema.scala | 52 +--- .../org/apache/spark/sql/json/JsonSuite.scala | 4 ++ .../apache/spark/sql/json/TestJsonData.scala| 9 3 files changed, 48 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0131142d/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala index 06aa19e..63764b8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala @@ -43,7 +43,7 @@ private[sql] object InferSchema { } // perform schema inference on each row and merge afterwards -schemaData.mapPartitions { iter = +val rootType = schemaData.mapPartitions { iter = val factory = new JsonFactory() iter.map { row = try { @@ -55,8 +55,13 @@ private[sql] object InferSchema { StructType(Seq(StructField(columnNameOfCorruptRecords, StringType))) } } -}.treeAggregate[DataType](StructType(Seq()))(compatibleRootType, compatibleRootType) match { - case st: StructType = nullTypeToStringType(st) +}.treeAggregate[DataType](StructType(Seq()))(compatibleRootType, compatibleRootType) + +canonicalizeType(rootType) match { + case Some(st: StructType) = st + case _ = +// canonicalizeType erases all empty structs, including the only one we want to keep +StructType(Seq()) } } @@ -116,22 +121,35 @@ private[sql] object InferSchema { } } - private def nullTypeToStringType(struct: StructType): StructType = { -val fields = struct.fields.map { - case StructField(fieldName, dataType, nullable, _) = -val newType = dataType match { - case NullType = StringType - case ArrayType(NullType, containsNull) = ArrayType(StringType, containsNull) - case ArrayType(struct: StructType, containsNull) = -ArrayType(nullTypeToStringType(struct), containsNull) - case struct: StructType = nullTypeToStringType(struct) - case other: DataType = other -} + /** + * Convert NullType to StringType and remove StructTypes with no fields + */ + private def canonicalizeType: DataType = Option[DataType] = { +case at@ArrayType(elementType, _) = + for { +canonicalType - canonicalizeType(elementType) + } yield { +at.copy(canonicalType) + } -StructField(fieldName, newType, nullable) -} +case StructType(fields) = + val canonicalFields = for { +field - fields +if field.name.nonEmpty +canonicalType - canonicalizeType(field.dataType) + } yield { +field.copy(dataType = canonicalType) + } + + if (canonicalFields.nonEmpty) { +Some(StructType(canonicalFields)) + } else { +// per SPARK-8093: empty structs should be deleted +None + } -StructType(fields) +case NullType = Some(StringType) +case other = Some(other) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/0131142d/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index f8d62f9..96ecf8f 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++
spark git commit: [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents
Repository: spark Updated Branches: refs/heads/master 1fa29c2df - 9814b971f [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents Author: Nathan Howell nhow...@godaddy.com Closes #6799 from NathanHowell/spark-8093 and squashes the following commits: 76ac3e8 [Nathan Howell] [SPARK-8093] [SQL] Remove empty structs inferred from JSON documents Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9814b971 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9814b971 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9814b971 Branch: refs/heads/master Commit: 9814b971f07dff8a99f1b8ad2adf70614f1c690b Parents: 1fa29c2 Author: Nathan Howell nhow...@godaddy.com Authored: Fri Jun 19 16:19:28 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 19 16:19:28 2015 -0700 -- .../org/apache/spark/sql/json/InferSchema.scala | 52 +--- .../org/apache/spark/sql/json/JsonSuite.scala | 4 ++ .../apache/spark/sql/json/TestJsonData.scala| 9 3 files changed, 48 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9814b971/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala index 565d102..afe2c6c 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/InferSchema.scala @@ -43,7 +43,7 @@ private[sql] object InferSchema { } // perform schema inference on each row and merge afterwards -schemaData.mapPartitions { iter = +val rootType = schemaData.mapPartitions { iter = val factory = new JsonFactory() iter.map { row = try { @@ -55,8 +55,13 @@ private[sql] object InferSchema { StructType(Seq(StructField(columnNameOfCorruptRecords, StringType))) } } -}.treeAggregate[DataType](StructType(Seq()))(compatibleRootType, compatibleRootType) match { - case st: StructType = nullTypeToStringType(st) +}.treeAggregate[DataType](StructType(Seq()))(compatibleRootType, compatibleRootType) + +canonicalizeType(rootType) match { + case Some(st: StructType) = st + case _ = +// canonicalizeType erases all empty structs, including the only one we want to keep +StructType(Seq()) } } @@ -116,22 +121,35 @@ private[sql] object InferSchema { } } - private def nullTypeToStringType(struct: StructType): StructType = { -val fields = struct.fields.map { - case StructField(fieldName, dataType, nullable, _) = -val newType = dataType match { - case NullType = StringType - case ArrayType(NullType, containsNull) = ArrayType(StringType, containsNull) - case ArrayType(struct: StructType, containsNull) = -ArrayType(nullTypeToStringType(struct), containsNull) - case struct: StructType = nullTypeToStringType(struct) - case other: DataType = other -} + /** + * Convert NullType to StringType and remove StructTypes with no fields + */ + private def canonicalizeType: DataType = Option[DataType] = { +case at@ArrayType(elementType, _) = + for { +canonicalType - canonicalizeType(elementType) + } yield { +at.copy(canonicalType) + } -StructField(fieldName, newType, nullable) -} +case StructType(fields) = + val canonicalFields = for { +field - fields +if field.name.nonEmpty +canonicalType - canonicalizeType(field.dataType) + } yield { +field.copy(dataType = canonicalType) + } + + if (canonicalFields.nonEmpty) { +Some(StructType(canonicalFields)) + } else { +// per SPARK-8093: empty structs should be deleted +None + } -StructType(fields) +case NullType = Some(StringType) +case other = Some(other) } /** http://git-wip-us.apache.org/repos/asf/spark/blob/9814b971/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala index 945d437..c32d9f8 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/JsonSuite.scala @@ -1103,4 +1103,8 @@ class JsonSuite extends QueryTest with TestJsonData { } } + test(SPARK-8093 Erase empty structs) { +val emptySchema =
spark git commit: [HOTFIX] Hotfix branch-1.4 building by removing avgMetrics in CrossValidatorSuite
Repository: spark Updated Branches: refs/heads/branch-1.4 2a7ea31a9 - b836bac3f [HOTFIX] Hotfix branch-1.4 building by removing avgMetrics in CrossValidatorSuite Ref. #6905 ping yhuai Author: Liang-Chi Hsieh vii...@gmail.com Closes #6929 from viirya/hot_fix_cv_test and squashes the following commits: b1aec53 [Liang-Chi Hsieh] Hotfix branch-1.4 by removing avgMetrics in CrossValidatorSuite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b836bac3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b836bac3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b836bac3 Branch: refs/heads/branch-1.4 Commit: b836bac3fe3e929ddef70851a1c93980ffa04820 Parents: 2a7ea31 Author: Liang-Chi Hsieh vii...@gmail.com Authored: Sun Jun 21 22:25:08 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun Jun 21 22:25:08 2015 -0700 -- .../scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala | 2 -- 1 file changed, 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b836bac3/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala -- diff --git a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala index d00b484..9a02c4f 100644 --- a/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala +++ b/mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala @@ -78,14 +78,12 @@ class CrossValidatorSuite extends SparkFunSuite with MLlibTestSparkContext { val parent = cvModel.bestModel.parent.asInstanceOf[LinearRegression] assert(parent.getRegParam === 0.001) assert(parent.getMaxIter === 10) -assert(cvModel.avgMetrics.length === lrParamMaps.length) eval.setMetricName(r2) val cvModel2 = cv.fit(dataset) val parent2 = cvModel2.bestModel.parent.asInstanceOf[LinearRegression] assert(parent2.getRegParam === 0.001) assert(parent2.getMaxIter === 10) -assert(cvModel2.avgMetrics.length === lrParamMaps.length) } test(validateParams should check estimatorParamMaps) { - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [HOT-FIX] Fix compilation (caused by 0131142d98b191f6cc112d383aa10582a3ac35bf)
Repository: spark Updated Branches: refs/heads/branch-1.4 0131142d9 - 2510365fa [HOT-FIX] Fix compilation (caused by 0131142d98b191f6cc112d383aa10582a3ac35bf) Author: Yin Huai yh...@databricks.com Closes #6913 from yhuai/branch-1.4-hotfix and squashes the following commits: 7f91fa0 [Yin Huai] [HOT-FIX] Fix compilation (caused by 0131142d98b191f6cc112d383aa10582a3ac35bf). Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2510365f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2510365f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2510365f Branch: refs/heads/branch-1.4 Commit: 2510365faaa025e6f2767506ff1726dfd0ed4a77 Parents: 0131142 Author: Yin Huai yh...@databricks.com Authored: Fri Jun 19 17:29:51 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 19 17:29:51 2015 -0700 -- .../src/test/scala/org/apache/spark/sql/json/TestJsonData.scala| 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2510365f/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala index 9322f17..a8dfd7e 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/json/TestJsonData.scala @@ -186,7 +186,7 @@ object TestJsonData { {b:str_b_4, a:str_a_4, c:str_c_4} :: ] :: Nil) - val emptyRecords: RDD[String] = + val emptyRecords = TestSQLContext.sparkContext.parallelize( { :: :: - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8498] [SQL] Add regression test for SPARK-8470
Repository: spark Updated Branches: refs/heads/branch-1.4 2510365fa - 2248ad8b7 [SPARK-8498] [SQL] Add regression test for SPARK-8470 **Summary of the problem in SPARK-8470.** When using `HiveContext` to create a data frame of a user case class, Spark throws `scala.reflect.internal.MissingRequirementError` when it tries to infer the schema using reflection. This is caused by `HiveContext` silently overwriting the context class loader containing the user classes. **What this issue is about.** This issue adds regression tests for SPARK-8470, which is already fixed in #6891. We closed SPARK-8470 as a duplicate because it is a different manifestation of the same problem in SPARK-8368. Due to the complexity of the reproduction, this requires us to pre-package a special test jar and include it in the Spark project itself. I tested this with and without the fix in #6891 and verified that it passes only if the fix is present. Author: Andrew Or and...@databricks.com Closes #6909 from andrewor14/SPARK-8498 and squashes the following commits: 5e9d688 [Andrew Or] Add regression test for SPARK-8470 (cherry picked from commit 093c34838d1db7a9375f36a9a2ab5d96a23ae683) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2248ad8b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2248ad8b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2248ad8b Branch: refs/heads/branch-1.4 Commit: 2248ad8b702669bc35a4df9dbc94b9d268e6fe9c Parents: 2510365 Author: Andrew Or and...@databricks.com Authored: Fri Jun 19 17:34:09 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 19 17:34:36 2015 -0700 -- .../regression-test-SPARK-8498/Main.scala | 43 +++ .../MyCoolClass.scala | 20 + .../regression-test-SPARK-8498/test.jar | Bin 0 - 6811 bytes .../spark/sql/hive/HiveSparkSubmitSuite.scala | 13 ++ 4 files changed, 76 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2248ad8b/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala -- diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala new file mode 100644 index 000..858dd6b --- /dev/null +++ b/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the License); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.hive.HiveContext + +/** + * Entry point in test application for SPARK-8498. + * + * This file is not meant to be compiled during tests. It is already included + * in a pre-built test.jar located in the same directory as this file. + * This is included here for reference only and should NOT be modified without + * rebuilding the test jar itself. + * + * This is used in org.apache.spark.sql.hive.HiveSparkSubmitSuite. + */ +object Main { + def main(args: Array[String]) { +println(Running regression test for SPARK-8498.) +val sc = new SparkContext(local, testing) +val hc = new HiveContext(sc) +// This line should not throw scala.reflect.internal.MissingRequirementError. +// See SPARK-8470 for more detail. +val df = hc.createDataFrame(Seq(MyCoolClass(1, 2, 3))) +df.collect() +println(Regression test for SPARK-8498 success!) + } +} + http://git-wip-us.apache.org/repos/asf/spark/blob/2248ad8b/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala -- diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala b/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala new file mode 100644 index 000..a72c063 --- /dev/null +++ b/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala @@ -0,0 +1,20 @@ +/* +
spark git commit: [SPARK-8498] [SQL] Add regression test for SPARK-8470
Repository: spark Updated Branches: refs/heads/master b305e377f - 093c34838 [SPARK-8498] [SQL] Add regression test for SPARK-8470 **Summary of the problem in SPARK-8470.** When using `HiveContext` to create a data frame of a user case class, Spark throws `scala.reflect.internal.MissingRequirementError` when it tries to infer the schema using reflection. This is caused by `HiveContext` silently overwriting the context class loader containing the user classes. **What this issue is about.** This issue adds regression tests for SPARK-8470, which is already fixed in #6891. We closed SPARK-8470 as a duplicate because it is a different manifestation of the same problem in SPARK-8368. Due to the complexity of the reproduction, this requires us to pre-package a special test jar and include it in the Spark project itself. I tested this with and without the fix in #6891 and verified that it passes only if the fix is present. Author: Andrew Or and...@databricks.com Closes #6909 from andrewor14/SPARK-8498 and squashes the following commits: 5e9d688 [Andrew Or] Add regression test for SPARK-8470 Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/093c3483 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/093c3483 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/093c3483 Branch: refs/heads/master Commit: 093c34838d1db7a9375f36a9a2ab5d96a23ae683 Parents: b305e37 Author: Andrew Or and...@databricks.com Authored: Fri Jun 19 17:34:09 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jun 19 17:34:09 2015 -0700 -- .../regression-test-SPARK-8498/Main.scala | 43 +++ .../MyCoolClass.scala | 20 + .../regression-test-SPARK-8498/test.jar | Bin 0 - 6811 bytes .../spark/sql/hive/HiveSparkSubmitSuite.scala | 13 ++ 4 files changed, 76 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/093c3483/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala -- diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala b/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala new file mode 100644 index 000..858dd6b --- /dev/null +++ b/sql/hive/src/test/resources/regression-test-SPARK-8498/Main.scala @@ -0,0 +1,43 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the License); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an AS IS BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +import org.apache.spark.{SparkConf, SparkContext} +import org.apache.spark.sql.hive.HiveContext + +/** + * Entry point in test application for SPARK-8498. + * + * This file is not meant to be compiled during tests. It is already included + * in a pre-built test.jar located in the same directory as this file. + * This is included here for reference only and should NOT be modified without + * rebuilding the test jar itself. + * + * This is used in org.apache.spark.sql.hive.HiveSparkSubmitSuite. + */ +object Main { + def main(args: Array[String]) { +println(Running regression test for SPARK-8498.) +val sc = new SparkContext(local, testing) +val hc = new HiveContext(sc) +// This line should not throw scala.reflect.internal.MissingRequirementError. +// See SPARK-8470 for more detail. +val df = hc.createDataFrame(Seq(MyCoolClass(1, 2, 3))) +df.collect() +println(Regression test for SPARK-8498 success!) + } +} + http://git-wip-us.apache.org/repos/asf/spark/blob/093c3483/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala -- diff --git a/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala b/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala new file mode 100644 index 000..a72c063 --- /dev/null +++ b/sql/hive/src/test/resources/regression-test-SPARK-8498/MyCoolClass.scala @@ -0,0 +1,20 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE
spark git commit: [SPARK-8567] [SQL] Add logs to record the progress of HiveSparkSubmitSuite.
Repository: spark Updated Branches: refs/heads/master e988adb58 - f9b397f54 [SPARK-8567] [SQL] Add logs to record the progress of HiveSparkSubmitSuite. Author: Yin Huai yh...@databricks.com Closes #7009 from yhuai/SPARK-8567 and squashes the following commits: 62fb1f9 [Yin Huai] Add sc.stop(). b22cf7d [Yin Huai] Add logs. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f9b397f5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f9b397f5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f9b397f5 Branch: refs/heads/master Commit: f9b397f54d1c491680d70aba210bb8211fd249c1 Parents: e988adb Author: Yin Huai yh...@databricks.com Authored: Thu Jun 25 06:52:03 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu Jun 25 06:52:03 2015 -0700 -- .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala| 9 + 1 file changed, 9 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f9b397f5/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index b875e52..a38ed23 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -115,6 +115,7 @@ object SparkSubmitClassLoaderTest extends Logging { val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i = (i, i))).toDF(i, j) +logInfo(Testing load classes at the driver side.) // First, we load classes at driver side. try { Class.forName(args(0), true, Thread.currentThread().getContextClassLoader) @@ -124,6 +125,7 @@ object SparkSubmitClassLoaderTest extends Logging { throw new Exception(Could not load user class from jar:\n, t) } // Second, we load classes at the executor side. +logInfo(Testing load classes at the executor side.) val result = df.mapPartitions { x = var exception: String = null try { @@ -141,6 +143,7 @@ object SparkSubmitClassLoaderTest extends Logging { } // Load a Hive UDF from the jar. +logInfo(Registering temporary Hive UDF provided in a jar.) hiveContext.sql( |CREATE TEMPORARY FUNCTION example_max @@ -150,18 +153,23 @@ object SparkSubmitClassLoaderTest extends Logging { hiveContext.createDataFrame((1 to 10).map(i = (i, sstr$i))).toDF(key, val) source.registerTempTable(sourceTable) // Load a Hive SerDe from the jar. +logInfo(Creating a Hive table with a SerDe provided in a jar.) hiveContext.sql( |CREATE TABLE t1(key int, val string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' .stripMargin) // Actually use the loaded UDF and SerDe. +logInfo(Writing data into the table.) hiveContext.sql( INSERT INTO TABLE t1 SELECT example_max(key) as key, val FROM sourceTable GROUP BY val) +logInfo(Running a simple query on the table.) val count = hiveContext.table(t1).orderBy(key, val).count() if (count != 10) { throw new Exception(stable t1 should have 10 rows instead of $count rows) } +logInfo(Test finishes.) +sc.stop() } } @@ -199,5 +207,6 @@ object SparkSQLConfTest extends Logging { val hiveContext = new TestHiveContext(sc) // Run a simple command to make sure all lazy vals in hiveContext get instantiated. hiveContext.tables().collect() +sc.stop() } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-6749] [SQL] Make metastore client robust to underlying socket connection loss
Repository: spark Updated Branches: refs/heads/master a458efc66 - 50c3a86f4 [SPARK-6749] [SQL] Make metastore client robust to underlying socket connection loss This works around a bug in the underlying RetryingMetaStoreClient (HIVE-10384) by refreshing the metastore client on thrift exceptions. We attempt to emulate the proper hive behavior by retrying only as configured by hiveconf. Author: Eric Liang e...@databricks.com Closes #6912 from ericl/spark-6749 and squashes the following commits: 2d54b55 [Eric Liang] use conf from state 0e3a74e [Eric Liang] use shim properly 980b3e5 [Eric Liang] Fix conf parsing hive 0.14 conf. 92459b6 [Eric Liang] Work around RetryingMetaStoreClient bug Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/50c3a86f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/50c3a86f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/50c3a86f Branch: refs/heads/master Commit: 50c3a86f42d7dfd1acbda65c1e5afbd3db1406df Parents: a458efc Author: Eric Liang e...@databricks.com Authored: Tue Jun 23 22:27:17 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue Jun 23 22:27:17 2015 -0700 -- .../spark/sql/hive/client/ClientWrapper.scala | 55 +++- .../apache/spark/sql/hive/client/HiveShim.scala | 19 +++ 2 files changed, 72 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/50c3a86f/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala index 42c2d4c..2f771d7 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/client/ClientWrapper.scala @@ -20,6 +20,7 @@ package org.apache.spark.sql.hive.client import java.io.{BufferedReader, InputStreamReader, File, PrintStream} import java.net.URI import java.util.{ArrayList = JArrayList, Map = JMap, List = JList, Set = JSet} +import javax.annotation.concurrent.GuardedBy import scala.collection.JavaConversions._ import scala.language.reflectiveCalls @@ -136,12 +137,62 @@ private[hive] class ClientWrapper( // TODO: should be a def?s // When we create this val client, the HiveConf of it (conf) is the one associated with state. - private val client = Hive.get(conf) + @GuardedBy(this) + private var client = Hive.get(conf) + + // We use hive's conf for compatibility. + private val retryLimit = conf.getIntVar(HiveConf.ConfVars.METASTORETHRIFTFAILURERETRIES) + private val retryDelayMillis = shim.getMetastoreClientConnectRetryDelayMillis(conf) + + /** + * Runs `f` with multiple retries in case the hive metastore is temporarily unreachable. + */ + private def retryLocked[A](f: = A): A = synchronized { +// Hive sometimes retries internally, so set a deadline to avoid compounding delays. +val deadline = System.nanoTime + (retryLimit * retryDelayMillis * 1e6).toLong +var numTries = 0 +var caughtException: Exception = null +do { + numTries += 1 + try { +return f + } catch { +case e: Exception if causedByThrift(e) = + caughtException = e + logWarning( +HiveClientWrapper got thrift exception, destroying client and retrying + + s(${retryLimit - numTries} tries remaining), e) + Thread.sleep(retryDelayMillis) + try { +client = Hive.get(state.getConf, true) + } catch { +case e: Exception if causedByThrift(e) = + logWarning(Failed to refresh hive client, will retry., e) + } + } +} while (numTries = retryLimit System.nanoTime deadline) +if (System.nanoTime deadline) { + logWarning(Deadline exceeded) +} +throw caughtException + } + + private def causedByThrift(e: Throwable): Boolean = { +var target = e +while (target != null) { + val msg = target.getMessage() + if (msg != null msg.matches((?s).*(TApplication|TProtocol|TTransport)Exception.*)) { +return true + } + target = target.getCause() +} +false + } /** * Runs `f` with ThreadLocal session state and classloaders configured for this version of hive. */ - private def withHiveState[A](f: = A): A = synchronized { + private def withHiveState[A](f: = A): A = retryLocked { val original = Thread.currentThread().getContextClassLoader // Set the thread local metastore client to the client associated with this ClientWrapper. Hive.set(client)
spark git commit: [SPARK-8567] [SQL] Debugging flaky HiveSparkSubmitSuite
Repository: spark Updated Branches: refs/heads/master cc465fd92 - 9d36ec243 [SPARK-8567] [SQL] Debugging flaky HiveSparkSubmitSuite Using similar approach used in `HiveThriftServer2Suite` to print stdout/stderr of the spawned process instead of logging them to see what happens on Jenkins. (This test suite only fails on Jenkins and doesn't spill out any log...) cc yhuai Author: Cheng Lian l...@databricks.com Closes #6978 from liancheng/debug-hive-spark-submit-suite and squashes the following commits: b031647 [Cheng Lian] Prints process stdout/stderr instead of logging them Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/9d36ec24 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/9d36ec24 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/9d36ec24 Branch: refs/heads/master Commit: 9d36ec24312f0a9865b4392f89e9611a5b80916d Parents: cc465fd Author: Cheng Lian l...@databricks.com Authored: Wed Jun 24 09:49:20 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed Jun 24 09:49:20 2015 -0700 -- .../apache/spark/sql/hive/HiveSparkSubmitSuite.scala | 14 +++--- 1 file changed, 11 insertions(+), 3 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/9d36ec24/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index ab44303..d85516a 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -19,6 +19,8 @@ package org.apache.spark.sql.hive import java.io.File +import scala.sys.process.{ProcessLogger, Process} + import org.apache.spark._ import org.apache.spark.sql.hive.test.{TestHive, TestHiveContext} import org.apache.spark.util.{ResetSystemProperties, Utils} @@ -82,12 +84,18 @@ class HiveSparkSubmitSuite // This is copied from org.apache.spark.deploy.SparkSubmitSuite private def runSparkSubmit(args: Seq[String]): Unit = { val sparkHome = sys.props.getOrElse(spark.test.home, fail(spark.test.home is not set!)) -val process = Utils.executeCommand( +val process = Process( Seq(./bin/spark-submit) ++ args, new File(sparkHome), - Map(SPARK_TESTING - 1, SPARK_HOME - sparkHome)) + SPARK_TESTING - 1, + SPARK_HOME - sparkHome +).run(ProcessLogger( + (line: String) = { println(sout $line) }, + (line: String) = { println(serr $line) } +)) + try { - val exitCode = failAfter(120 seconds) { process.waitFor() } + val exitCode = failAfter(120 seconds) { process.exitValue() } if (exitCode != 0) { fail(sProcess returned with exit code $exitCode. See the log4j logs for more detail.) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8578] [SQL] Should ignore user defined output committer when appending data (branch 1.4)
Repository: spark Updated Branches: refs/heads/branch-1.4 eafbe1345 - 7e53ff258 [SPARK-8578] [SQL] Should ignore user defined output committer when appending data (branch 1.4) This is https://github.com/apache/spark/pull/6964 for branch 1.4. Author: Yin Huai yh...@databricks.com Closes #6966 from yhuai/SPARK-8578-branch-1.4 and squashes the following commits: 9c3947b [Yin Huai] Do not use a custom output commiter when appendiing data. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/7e53ff25 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/7e53ff25 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/7e53ff25 Branch: refs/heads/branch-1.4 Commit: 7e53ff25813dc6a79f728c91e6c1d4d4dfa32aab Parents: eafbe13 Author: Yin Huai yh...@databricks.com Authored: Wed Jun 24 09:51:18 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed Jun 24 09:51:18 2015 -0700 -- .../org/apache/spark/sql/sources/commands.scala | 89 .../sql/sources/hadoopFsRelationSuites.scala| 83 +- 2 files changed, 136 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/7e53ff25/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 9a75dd7..29a47f5 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -97,7 +97,8 @@ private[sql] case class InsertIntoHadoopFsRelation( val fs = outputPath.getFileSystem(hadoopConf) val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) -val doInsertion = (mode, fs.exists(qualifiedOutputPath)) match { +val pathExists = fs.exists(qualifiedOutputPath) +val doInsertion = (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) = sys.error(spath $qualifiedOutputPath already exists.) case (SaveMode.Overwrite, true) = @@ -108,6 +109,8 @@ private[sql] case class InsertIntoHadoopFsRelation( case (SaveMode.Ignore, exists) = !exists } +// If we are appending data to an existing dir. +val isAppend = (pathExists) (mode == SaveMode.Append) if (doInsertion) { val job = new Job(hadoopConf) @@ -133,10 +136,10 @@ private[sql] case class InsertIntoHadoopFsRelation( val partitionColumns = relation.partitionColumns.fieldNames if (partitionColumns.isEmpty) { -insert(new DefaultWriterContainer(relation, job), df) +insert(new DefaultWriterContainer(relation, job, isAppend), df) } else { val writerContainer = new DynamicPartitionWriterContainer( - relation, job, partitionColumns, PartitioningUtils.DEFAULT_PARTITION_NAME) + relation, job, partitionColumns, PartitioningUtils.DEFAULT_PARTITION_NAME, isAppend) insertWithDynamicPartitions(sqlContext, writerContainer, df, partitionColumns) } } @@ -286,7 +289,8 @@ private[sql] case class InsertIntoHadoopFsRelation( private[sql] abstract class BaseWriterContainer( @transient val relation: HadoopFsRelation, -@transient job: Job) +@transient job: Job, +isAppend: Boolean) extends SparkHadoopMapReduceUtil with Logging with Serializable { @@ -365,34 +369,47 @@ private[sql] abstract class BaseWriterContainer( } private def newOutputCommitter(context: TaskAttemptContext): OutputCommitter = { -val committerClass = context.getConfiguration.getClass( - SQLConf.OUTPUT_COMMITTER_CLASS, null, classOf[OutputCommitter]) - -Option(committerClass).map { clazz = - logInfo(sUsing user defined output committer class ${clazz.getCanonicalName}) - - // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat - // has an associated output committer. To override this output committer, - // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. - // If a data source needs to override the output committer, it needs to set the - // output committer in prepareForWrite method. - if (classOf[MapReduceFileOutputCommitter].isAssignableFrom(clazz)) { -// The specified output committer is a FileOutputCommitter. -// So, we will use the FileOutputCommitter-specified constructor. -val ctor = clazz.getDeclaredConstructor(classOf[Path], classOf[TaskAttemptContext]) -ctor.newInstance(new Path(outputPath), context) - } else { -// The specified output committer is just
spark git commit: [SPARK-8578] [SQL] Should ignore user defined output committer when appending data
Repository: spark Updated Branches: refs/heads/master 9d36ec243 - bba6699d0 [SPARK-8578] [SQL] Should ignore user defined output committer when appending data https://issues.apache.org/jira/browse/SPARK-8578 It is not very safe to use a custom output committer when append data to an existing dir. This changes adds the logic to check if we are appending data, and if so, we use the output committer associated with the file output format. Author: Yin Huai yh...@databricks.com Closes #6964 from yhuai/SPARK-8578 and squashes the following commits: 43544c4 [Yin Huai] Do not use a custom output commiter when appendiing data. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bba6699d Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bba6699d Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bba6699d Branch: refs/heads/master Commit: bba6699d0e9093bc041a9a33dd31992790f32174 Parents: 9d36ec2 Author: Yin Huai yh...@databricks.com Authored: Wed Jun 24 09:50:03 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed Jun 24 09:50:03 2015 -0700 -- .../org/apache/spark/sql/sources/commands.scala | 89 .../sql/sources/hadoopFsRelationSuites.scala| 83 +- 2 files changed, 136 insertions(+), 36 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bba6699d/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala index 215e53c..fb6173f 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/commands.scala @@ -96,7 +96,8 @@ private[sql] case class InsertIntoHadoopFsRelation( val fs = outputPath.getFileSystem(hadoopConf) val qualifiedOutputPath = outputPath.makeQualified(fs.getUri, fs.getWorkingDirectory) -val doInsertion = (mode, fs.exists(qualifiedOutputPath)) match { +val pathExists = fs.exists(qualifiedOutputPath) +val doInsertion = (mode, pathExists) match { case (SaveMode.ErrorIfExists, true) = sys.error(spath $qualifiedOutputPath already exists.) case (SaveMode.Overwrite, true) = @@ -107,6 +108,8 @@ private[sql] case class InsertIntoHadoopFsRelation( case (SaveMode.Ignore, exists) = !exists } +// If we are appending data to an existing dir. +val isAppend = (pathExists) (mode == SaveMode.Append) if (doInsertion) { val job = new Job(hadoopConf) @@ -130,10 +133,10 @@ private[sql] case class InsertIntoHadoopFsRelation( val partitionColumns = relation.partitionColumns.fieldNames if (partitionColumns.isEmpty) { -insert(new DefaultWriterContainer(relation, job), df) +insert(new DefaultWriterContainer(relation, job, isAppend), df) } else { val writerContainer = new DynamicPartitionWriterContainer( - relation, job, partitionColumns, PartitioningUtils.DEFAULT_PARTITION_NAME) + relation, job, partitionColumns, PartitioningUtils.DEFAULT_PARTITION_NAME, isAppend) insertWithDynamicPartitions(sqlContext, writerContainer, df, partitionColumns) } } @@ -277,7 +280,8 @@ private[sql] case class InsertIntoHadoopFsRelation( private[sql] abstract class BaseWriterContainer( @transient val relation: HadoopFsRelation, -@transient job: Job) +@transient job: Job, +isAppend: Boolean) extends SparkHadoopMapReduceUtil with Logging with Serializable { @@ -356,34 +360,47 @@ private[sql] abstract class BaseWriterContainer( } private def newOutputCommitter(context: TaskAttemptContext): OutputCommitter = { -val committerClass = context.getConfiguration.getClass( - SQLConf.OUTPUT_COMMITTER_CLASS.key, null, classOf[OutputCommitter]) - -Option(committerClass).map { clazz = - logInfo(sUsing user defined output committer class ${clazz.getCanonicalName}) - - // Every output format based on org.apache.hadoop.mapreduce.lib.output.OutputFormat - // has an associated output committer. To override this output committer, - // we will first try to use the output committer set in SQLConf.OUTPUT_COMMITTER_CLASS. - // If a data source needs to override the output committer, it needs to set the - // output committer in prepareForWrite method. - if (classOf[MapReduceFileOutputCommitter].isAssignableFrom(clazz)) { -// The specified output committer is a FileOutputCommitter. -// So, we will use the FileOutputCommitter-specified constructor. -val ctor
spark git commit: [SPARK-7859] [SQL] Collect_set() behavior differences which fails the unit test under jdk8
Repository: spark Updated Branches: refs/heads/branch-1.4 994abbaeb - d73900a90 [SPARK-7859] [SQL] Collect_set() behavior differences which fails the unit test under jdk8 To reproduce that: ``` JAVA_HOME=/home/hcheng/Java/jdk1.8.0_45 | build/sbt -Phadoop-2.3 -Phive 'test-only org.apache.spark.sql.hive.execution.HiveWindowFunctionQueryWithoutCodeGenSuite' ``` A simple workaround to fix that is update the original query, for getting the output size instead of the exact elements of the array (output by collect_set()) Author: Cheng Hao hao.ch...@intel.com Closes #6402 from chenghao-intel/windowing and squashes the following commits: 99312ad [Cheng Hao] add order by for the select clause edf8ce3 [Cheng Hao] update the code as suggested 7062da7 [Cheng Hao] fix the collect_set() behaviour differences under different versions of JDK (cherry picked from commit 13321e65559f6354ec1287a690580fd6f498ef89) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d73900a9 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d73900a9 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d73900a9 Branch: refs/heads/branch-1.4 Commit: d73900a9034b22191e1440b18ee63b1835f09582 Parents: 994abba Author: Cheng Hao hao.ch...@intel.com Authored: Mon Jun 22 20:04:49 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 22 20:05:00 2015 -0700 -- .../HiveWindowFunctionQuerySuite.scala | 8 ++ ... testSTATs-0-6dfcd7925fb267699c4bf82737d4609 | 97 ...testSTATs-0-da0e0cca69e42118a96b8609b8fa5838 | 26 -- 3 files changed, 105 insertions(+), 26 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d73900a9/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala -- diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index 934452f..31a49a3 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -526,8 +526,14 @@ abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with | rows between 2 preceding and 2 following); .stripMargin, reset = false) + // collect_set() output array in an arbitrary order, hence causes different result + // when running this test suite under Java 7 and 8. + // We change the original sql query a little bit for making the test suite passed + // under different JDK createQueryTest(windowing.q -- 20. testSTATs, + |select p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp + |from ( |select p_mfgr,p_name, p_size, |stddev(p_retailprice) over w1 as sdev, |stddev_pop(p_retailprice) over w1 as sdev_pop, @@ -538,6 +544,8 @@ abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with |from part |window w1 as (distribute by p_mfgr sort by p_mfgr, p_name | rows between 2 preceding and 2 following) + |) t lateral view explode(uniq_size) d as uniq_data + |order by p_mfgr,p_name, p_size, sdev, sdev_pop, uniq_data, var, cor, covarp .stripMargin, reset = false) createQueryTest(windowing.q -- 21. testDISTs, http://git-wip-us.apache.org/repos/asf/spark/blob/d73900a9/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609 -- diff --git a/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609 b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609 new file mode 100644 index 000..7e5fcee --- /dev/null +++ b/sql/hive/src/test/resources/golden/windowing.q -- 20. testSTATs-0-6dfcd7925fb267699c4bf82737d4609 @@ -0,0 +1,97 @@ +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 2 66619.10876874991 0.811328754177887 2801.70745 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 6 66619.10876874991 0.811328754177887 2801.70745 +Manufacturer#1 almond antique burnished rose metallic 2 258.10677784349235 258.10677784349235 34 66619.10876874991
spark git commit: [SPARK-7853] [SQL] Fix HiveContext in Spark Shell
Repository: spark Updated Branches: refs/heads/master 0077af22c - 572b62caf [SPARK-7853] [SQL] Fix HiveContext in Spark Shell https://issues.apache.org/jira/browse/SPARK-7853 This fixes the problem introduced by my change in https://github.com/apache/spark/pull/6435, which causes that Hive Context fails to create in spark shell because of the class loader issue. Author: Yin Huai yh...@databricks.com Closes #6459 from yhuai/SPARK-7853 and squashes the following commits: 37ad33e [Yin Huai] Do not use hiveQlTable at all. 47cdb6d [Yin Huai] Move hiveconf.set to the end of setConf. 005649b [Yin Huai] Update comment. 35d86f3 [Yin Huai] Access TTable directly to make sure Hive will not internally use any metastore utility functions. 3737766 [Yin Huai] Recursively find all jars. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/572b62ca Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/572b62ca Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/572b62ca Branch: refs/heads/master Commit: 572b62cafe4bc7b1d464c9dcfb449c9d53456826 Parents: 0077af2 Author: Yin Huai yh...@databricks.com Authored: Thu May 28 17:12:30 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 28 17:12:30 2015 -0700 -- .../org/apache/spark/sql/hive/HiveContext.scala | 35 +++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 +++ 2 files changed, 25 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/572b62ca/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 9ab98fd..2ed71d3 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 @@ -189,24 +189,22 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS + sor change $HIVE_METASTORE_VERSION to $hiveExecutionVersion.) } - // We recursively add all jars in the class loader chain, - // starting from the given urlClassLoader. - def addJars(urlClassLoader: URLClassLoader): Array[URL] = { -val jarsInParent = urlClassLoader.getParent match { - case parent: URLClassLoader = addJars(parent) - case other = Array.empty[URL] -} -urlClassLoader.getURLs ++ jarsInParent + // We recursively find all jars in the class loader chain, + // starting from the given classLoader. + def allJars(classLoader: ClassLoader): Array[URL] = classLoader match { +case null = Array.empty[URL] +case urlClassLoader: URLClassLoader = + urlClassLoader.getURLs ++ allJars(urlClassLoader.getParent) +case other = allJars(other.getParent) } - val jars = Utils.getContextOrSparkClassLoader match { -case urlClassLoader: URLClassLoader = addJars(urlClassLoader) -case other = - throw new IllegalArgumentException( -Unable to locate hive jars to connect to metastore + -susing classloader ${other.getClass.getName}. + -Please set spark.sql.hive.metastore.jars) + val classLoader = Utils.getContextOrSparkClassLoader + val jars = allJars(classLoader) + if (jars.length == 0) { +throw new IllegalArgumentException( + Unable to locate hive jars to connect to metastore. + +Please set spark.sql.hive.metastore.jars.) } logInfo( @@ -356,9 +354,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def setConf(key: String, value: String): Unit = { super.setConf(key, value) -hiveconf.set(key, value) executionHive.runSqlHive(sSET $key=$value) metadataHive.runSqlHive(sSET $key=$value) +// If users put any Spark SQL setting in the spark conf (e.g. spark-defaults.conf), +// this setConf will be called in the constructor of the SQLContext. +// Also, calling hiveconf will create a default session containing a HiveConf, which +// will interfer with the creation of executionHive (which is a lazy val). So, +// we put hiveconf.set at the end of this method. +hiveconf.set(key, value) } /* A catalyst metadata catalog that points to the Hive Metastore. */ http://git-wip-us.apache.org/repos/asf/spark/blob/572b62ca/sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveMetastoreCatalog.scala -- diff --git a/sql/hive/src/main/scala
spark git commit: [SPARK-7853] [SQL] Fix HiveContext in Spark Shell
Repository: spark Updated Branches: refs/heads/branch-1.4 9c2c6b4a6 - 8f4a86eaa [SPARK-7853] [SQL] Fix HiveContext in Spark Shell https://issues.apache.org/jira/browse/SPARK-7853 This fixes the problem introduced by my change in https://github.com/apache/spark/pull/6435, which causes that Hive Context fails to create in spark shell because of the class loader issue. Author: Yin Huai yh...@databricks.com Closes #6459 from yhuai/SPARK-7853 and squashes the following commits: 37ad33e [Yin Huai] Do not use hiveQlTable at all. 47cdb6d [Yin Huai] Move hiveconf.set to the end of setConf. 005649b [Yin Huai] Update comment. 35d86f3 [Yin Huai] Access TTable directly to make sure Hive will not internally use any metastore utility functions. 3737766 [Yin Huai] Recursively find all jars. (cherry picked from commit 572b62cafe4bc7b1d464c9dcfb449c9d53456826) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8f4a86ea Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8f4a86ea Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8f4a86ea Branch: refs/heads/branch-1.4 Commit: 8f4a86eaa1cad9a2a7607fd5446105c93e5e424e Parents: 9c2c6b4 Author: Yin Huai yh...@databricks.com Authored: Thu May 28 17:12:30 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 28 17:12:38 2015 -0700 -- .../org/apache/spark/sql/hive/HiveContext.scala | 35 +++- .../spark/sql/hive/HiveMetastoreCatalog.scala | 12 +++ 2 files changed, 25 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8f4a86ea/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 9ab98fd..2ed71d3 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 @@ -189,24 +189,22 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { Specify a vaild path to the correct hive jars using $HIVE_METASTORE_JARS + sor change $HIVE_METASTORE_VERSION to $hiveExecutionVersion.) } - // We recursively add all jars in the class loader chain, - // starting from the given urlClassLoader. - def addJars(urlClassLoader: URLClassLoader): Array[URL] = { -val jarsInParent = urlClassLoader.getParent match { - case parent: URLClassLoader = addJars(parent) - case other = Array.empty[URL] -} -urlClassLoader.getURLs ++ jarsInParent + // We recursively find all jars in the class loader chain, + // starting from the given classLoader. + def allJars(classLoader: ClassLoader): Array[URL] = classLoader match { +case null = Array.empty[URL] +case urlClassLoader: URLClassLoader = + urlClassLoader.getURLs ++ allJars(urlClassLoader.getParent) +case other = allJars(other.getParent) } - val jars = Utils.getContextOrSparkClassLoader match { -case urlClassLoader: URLClassLoader = addJars(urlClassLoader) -case other = - throw new IllegalArgumentException( -Unable to locate hive jars to connect to metastore + -susing classloader ${other.getClass.getName}. + -Please set spark.sql.hive.metastore.jars) + val classLoader = Utils.getContextOrSparkClassLoader + val jars = allJars(classLoader) + if (jars.length == 0) { +throw new IllegalArgumentException( + Unable to locate hive jars to connect to metastore. + +Please set spark.sql.hive.metastore.jars.) } logInfo( @@ -356,9 +354,14 @@ class HiveContext(sc: SparkContext) extends SQLContext(sc) { override def setConf(key: String, value: String): Unit = { super.setConf(key, value) -hiveconf.set(key, value) executionHive.runSqlHive(sSET $key=$value) metadataHive.runSqlHive(sSET $key=$value) +// If users put any Spark SQL setting in the spark conf (e.g. spark-defaults.conf), +// this setConf will be called in the constructor of the SQLContext. +// Also, calling hiveconf will create a default session containing a HiveConf, which +// will interfer with the creation of executionHive (which is a lazy val). So, +// we put hiveconf.set at the end of this method. +hiveconf.set(key, value) } /* A catalyst metadata catalog that points to the Hive Metastore. */ http://git-wip-us.apache.org/repos/asf/spark/blob/8f4a86ea/sql/hive/src/main/scala/org/apache/spark/sql/hive
spark git commit: [SPARK-7847] [SQL] Fixes dynamic partition directory escaping
Repository: spark Updated Branches: refs/heads/branch-1.4 90525c9ba - a25ce91f9 [SPARK-7847] [SQL] Fixes dynamic partition directory escaping Please refer to [SPARK-7847] [1] for details. [1]: https://issues.apache.org/jira/browse/SPARK-7847 Author: Cheng Lian l...@databricks.com Closes #6389 from liancheng/spark-7847 and squashes the following commits: 935c652 [Cheng Lian] Adds test case for writing various data types as dynamic partition value f4fc398 [Cheng Lian] Converts partition columns to Scala type when writing dynamic partitions d0aeca0 [Cheng Lian] Fixes dynamic partition directory escaping (cherry picked from commit 15459db4f6867e95076cf53fade2fca833c4cf4e) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a25ce91f Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a25ce91f Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a25ce91f Branch: refs/heads/branch-1.4 Commit: a25ce91f9685604cfb567a6860182ba467ceed8d Parents: 90525c9 Author: Cheng Lian l...@databricks.com Authored: Wed May 27 10:09:12 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 10:09:20 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 22 -- .../spark/sql/sources/PartitioningUtils.scala | 76 +++- .../org/apache/spark/sql/sources/commands.scala | 57 ++- .../ParquetPartitionDiscoverySuite.scala| 57 ++- 4 files changed, 152 insertions(+), 60 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a25ce91f/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index cb1e608..8b3e1b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.parquet +import java.net.URI import java.util.{List = JList} import scala.collection.JavaConversions._ @@ -282,21 +283,28 @@ private[sql] class ParquetRelation2( val cacheMetadata = useMetadataCache @transient val cachedStatuses = inputFiles.map { f = - // In order to encode the authority of a Path containing special characters such as /, - // we need to use the string returned by the URI of the path to create a new Path. - val pathWithAuthority = new Path(f.getPath.toUri.toString) - + // In order to encode the authority of a Path containing special characters such as '/' + // (which does happen in some S3N credentials), we need to use the string returned by the + // URI of the path to create a new Path. + val pathWithEscapedAuthority = escapePathUserInfo(f.getPath) new FileStatus( f.getLen, f.isDir, f.getReplication, f.getBlockSize, f.getModificationTime, -f.getAccessTime, f.getPermission, f.getOwner, f.getGroup, pathWithAuthority) +f.getAccessTime, f.getPermission, f.getOwner, f.getGroup, pathWithEscapedAuthority) }.toSeq @transient val cachedFooters = footers.map { f = // In order to encode the authority of a Path containing special characters such as /, // we need to use the string returned by the URI of the path to create a new Path. - new Footer(new Path(f.getFile.toUri.toString), f.getParquetMetadata) + new Footer(escapePathUserInfo(f.getFile), f.getParquetMetadata) }.toSeq +private def escapePathUserInfo(path: Path): Path = { + val uri = path.toUri + new Path(new URI( +uri.getScheme, uri.getRawUserInfo, uri.getHost, uri.getPort, uri.getPath, +uri.getQuery, uri.getFragment)) +} + // Overridden so we can inject our own cached files statuses. override def getPartitions: Array[SparkPartition] = { val inputFormat = if (cacheMetadata) { @@ -377,7 +385,7 @@ private[sql] class ParquetRelation2( .orElse(readSchema()) .orElse(maybeMetastoreSchema) .getOrElse(sys.error(Failed to get the schema.)) - + // If this Parquet relation is converted from a Hive Metastore table, must reconcile case // case insensitivity issue and possible schema mismatch (probably caused by schema // evolution). http://git-wip-us.apache.org/repos/asf/spark/blob/a25ce91f/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala
spark git commit: [SPARK-7847] [SQL] Fixes dynamic partition directory escaping
Repository: spark Updated Branches: refs/heads/master ff0ddff46 - 15459db4f [SPARK-7847] [SQL] Fixes dynamic partition directory escaping Please refer to [SPARK-7847] [1] for details. [1]: https://issues.apache.org/jira/browse/SPARK-7847 Author: Cheng Lian l...@databricks.com Closes #6389 from liancheng/spark-7847 and squashes the following commits: 935c652 [Cheng Lian] Adds test case for writing various data types as dynamic partition value f4fc398 [Cheng Lian] Converts partition columns to Scala type when writing dynamic partitions d0aeca0 [Cheng Lian] Fixes dynamic partition directory escaping Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/15459db4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/15459db4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/15459db4 Branch: refs/heads/master Commit: 15459db4f6867e95076cf53fade2fca833c4cf4e Parents: ff0ddff Author: Cheng Lian l...@databricks.com Authored: Wed May 27 10:09:12 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 10:09:12 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 22 -- .../spark/sql/sources/PartitioningUtils.scala | 76 +++- .../org/apache/spark/sql/sources/commands.scala | 57 ++- .../ParquetPartitionDiscoverySuite.scala| 57 ++- 4 files changed, 152 insertions(+), 60 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/15459db4/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index cb1e608..8b3e1b2 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -17,6 +17,7 @@ package org.apache.spark.sql.parquet +import java.net.URI import java.util.{List = JList} import scala.collection.JavaConversions._ @@ -282,21 +283,28 @@ private[sql] class ParquetRelation2( val cacheMetadata = useMetadataCache @transient val cachedStatuses = inputFiles.map { f = - // In order to encode the authority of a Path containing special characters such as /, - // we need to use the string returned by the URI of the path to create a new Path. - val pathWithAuthority = new Path(f.getPath.toUri.toString) - + // In order to encode the authority of a Path containing special characters such as '/' + // (which does happen in some S3N credentials), we need to use the string returned by the + // URI of the path to create a new Path. + val pathWithEscapedAuthority = escapePathUserInfo(f.getPath) new FileStatus( f.getLen, f.isDir, f.getReplication, f.getBlockSize, f.getModificationTime, -f.getAccessTime, f.getPermission, f.getOwner, f.getGroup, pathWithAuthority) +f.getAccessTime, f.getPermission, f.getOwner, f.getGroup, pathWithEscapedAuthority) }.toSeq @transient val cachedFooters = footers.map { f = // In order to encode the authority of a Path containing special characters such as /, // we need to use the string returned by the URI of the path to create a new Path. - new Footer(new Path(f.getFile.toUri.toString), f.getParquetMetadata) + new Footer(escapePathUserInfo(f.getFile), f.getParquetMetadata) }.toSeq +private def escapePathUserInfo(path: Path): Path = { + val uri = path.toUri + new Path(new URI( +uri.getScheme, uri.getRawUserInfo, uri.getHost, uri.getPort, uri.getPath, +uri.getQuery, uri.getFragment)) +} + // Overridden so we can inject our own cached files statuses. override def getPartitions: Array[SparkPartition] = { val inputFormat = if (cacheMetadata) { @@ -377,7 +385,7 @@ private[sql] class ParquetRelation2( .orElse(readSchema()) .orElse(maybeMetastoreSchema) .getOrElse(sys.error(Failed to get the schema.)) - + // If this Parquet relation is converted from a Hive Metastore table, must reconcile case // case insensitivity issue and possible schema mismatch (probably caused by schema // evolution). http://git-wip-us.apache.org/repos/asf/spark/blob/15459db4/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala -- diff --git
spark git commit: [SPARK-7790] [SQL] date and decimal conversion for dynamic partition key
Repository: spark Updated Branches: refs/heads/master 6fec1a940 - 8161562ea [SPARK-7790] [SQL] date and decimal conversion for dynamic partition key Author: Daoyuan Wang daoyuan.w...@intel.com Closes #6318 from adrian-wang/dynpart and squashes the following commits: ad73b61 [Daoyuan Wang] not use sqlTestUtils for try catch because dont have sqlcontext here 6c33b51 [Daoyuan Wang] fix according to liancheng f0f8074 [Daoyuan Wang] some specific types as dynamic partition Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8161562e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8161562e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8161562e Branch: refs/heads/master Commit: 8161562eabc1eff430cfd9d8eaf413a8c4ef2cfb Parents: 6fec1a9 Author: Daoyuan Wang daoyuan.w...@intel.com Authored: Wed May 27 12:42:13 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 12:42:13 2015 -0700 -- .../hive/execution/InsertIntoHiveTable.scala| 2 +- .../spark/sql/hive/hiveWriterContainers.scala | 17 -- .../sql/hive/execution/SQLQuerySuite.scala | 33 3 files changed, 48 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8161562e/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala index c0b0b10..7a6ca48 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveTable.scala @@ -106,7 +106,7 @@ case class InsertIntoHiveTable( } writerContainer - .getLocalFileWriter(row) + .getLocalFileWriter(row, table.schema) .write(serializer.serialize(outputData, standardOI)) } http://git-wip-us.apache.org/repos/asf/spark/blob/8161562e/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala -- diff --git a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala index cbc381c..50b209f 100644 --- a/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala +++ b/sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala @@ -34,8 +34,10 @@ import org.apache.hadoop.hive.common.FileUtils import org.apache.spark.mapred.SparkHadoopMapRedUtil import org.apache.spark.sql.Row import org.apache.spark.{Logging, SerializableWritable, SparkHadoopWriter} +import org.apache.spark.sql.catalyst.util.DateUtils import org.apache.spark.sql.hive.{ShimFileSinkDesc = FileSinkDesc} import org.apache.spark.sql.hive.HiveShim._ +import org.apache.spark.sql.types._ /** * Internal helper class that saves an RDD using a Hive OutputFormat. @@ -92,7 +94,7 @@ private[hive] class SparkHiveWriterContainer( part- + numberFormat.format(splitID) + extension } - def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = writer + def getLocalFileWriter(row: Row, schema: StructType): FileSinkOperator.RecordWriter = writer def close() { // Seems the boolean value passed into close does not matter. @@ -195,11 +197,20 @@ private[spark] class SparkHiveDynamicPartitionWriterContainer( jobConf.setBoolean(SUCCESSFUL_JOB_OUTPUT_DIR_MARKER, oldMarker) } - override def getLocalFileWriter(row: Row): FileSinkOperator.RecordWriter = { + override def getLocalFileWriter(row: Row, schema: StructType): FileSinkOperator.RecordWriter = { +def convertToHiveRawString(col: String, value: Any): String = { + val raw = String.valueOf(value) + schema(col).dataType match { +case DateType = DateUtils.toString(raw.toInt) +case _: DecimalType = BigDecimal(raw).toString() +case _ = raw + } +} + val dynamicPartPath = dynamicPartColNames .zip(row.toSeq.takeRight(dynamicPartColNames.length)) .map { case (col, rawVal) = -val string = if (rawVal == null) null else String.valueOf(rawVal) +val string = if (rawVal == null) null else convertToHiveRawString(col, rawVal) val colString = if (string == null || string.isEmpty) { defaultPartName http://git-wip-us.apache.org/repos/asf/spark/blob/8161562e/sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/SQLQuerySuite.scala
spark git commit: [SPARK-7684] [SQL] Refactoring MetastoreDataSourcesSuite to workaround SPARK-7684
Repository: spark Updated Branches: refs/heads/branch-1.4 d33142fd8 - 89fe93fc3 [SPARK-7684] [SQL] Refactoring MetastoreDataSourcesSuite to workaround SPARK-7684 As stated in SPARK-7684, currently `TestHive.reset` has some execution order specific bug, which makes running specific test suites locally pretty frustrating. This PR refactors `MetastoreDataSourcesSuite` (which relies on `TestHive.reset` heavily) using various `withXxx` utility methods in `SQLTestUtils` to ask each test case to cleanup their own mess so that we can avoid calling `TestHive.reset`. Author: Cheng Lian l...@databricks.com Author: Yin Huai yh...@databricks.com Closes #6353 from liancheng/workaround-spark-7684 and squashes the following commits: 26939aa [Yin Huai] Move the initialization of jsonFilePath to beforeAll. a423d48 [Cheng Lian] Fixes Scala style issue dfe45d0 [Cheng Lian] Refactors MetastoreDataSourcesSuite to workaround SPARK-7684 92a116d [Cheng Lian] Fixes minor styling issues (cherry picked from commit b97ddff000b99adca3dd8fe13d01054fd5014fa0) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/89fe93fc Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/89fe93fc Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/89fe93fc Branch: refs/heads/branch-1.4 Commit: 89fe93fc3b93009f1741b59dda6a4a9005128d1e Parents: d33142f Author: Cheng Lian l...@databricks.com Authored: Wed May 27 13:09:33 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 13:09:42 2015 -0700 -- .../scala/org/apache/spark/sql/QueryTest.scala |4 + .../apache/spark/sql/test/SQLTestUtils.scala| 12 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 1372 +- 3 files changed, 722 insertions(+), 666 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/89fe93fc/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 bbf9ab1..98ba3c9 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 @@ -67,6 +67,10 @@ class QueryTest extends PlanTest { checkAnswer(df, Seq(expectedAnswer)) } + protected def checkAnswer(df: DataFrame, expectedAnswer: DataFrame): Unit = { +checkAnswer(df, expectedAnswer.collect()) + } + def sqlTest(sqlString: String, expectedAnswer: Seq[Row])(implicit sqlContext: SQLContext) { test(sqlString) { checkAnswer(sqlContext.sql(sqlString), expectedAnswer) http://git-wip-us.apache.org/repos/asf/spark/blob/89fe93fc/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala index ca66cdc..17a8b0c 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/test/SQLTestUtils.scala @@ -75,14 +75,18 @@ trait SQLTestUtils { /** * Drops temporary table `tableName` after calling `f`. */ - protected def withTempTable(tableName: String)(f: = Unit): Unit = { -try f finally sqlContext.dropTempTable(tableName) + protected def withTempTable(tableNames: String*)(f: = Unit): Unit = { +try f finally tableNames.foreach(sqlContext.dropTempTable) } /** * Drops table `tableName` after calling `f`. */ - protected def withTable(tableName: String)(f: = Unit): Unit = { -try f finally sqlContext.sql(sDROP TABLE IF EXISTS $tableName) + protected def withTable(tableNames: String*)(f: = Unit): Unit = { +try f finally { + tableNames.foreach { name = +sqlContext.sql(sDROP TABLE IF EXISTS $name) + } +} } } http://git-wip-us.apache.org/repos/asf/spark/blob/89fe93fc/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala index 9623ef0..58e2d1f 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala @@ -21,770 +21,818 @@ import java.io.File import scala.collection.mutable.ArrayBuffer +import org.scalatest.BeforeAndAfterAll + import
spark git commit: [SPARK-7853] [SQL] Fixes a class loader issue in Spark SQL
Repository: spark Updated Branches: refs/heads/branch-1.4 89fe93fc3 - e07b71560 [SPARK-7853] [SQL] Fixes a class loader issue in Spark SQL This PR is based on PR #6396 authored by chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes. yhuai helped updating the test case, and I fixed a bug in the original `CliSuite`: while testing the CLI tool with `runCliWithin`, we don't append `\n` to the last query, thus the last query is never executed. Original PR description is pasted below. ``` bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; ``` Throws exception like ``` 15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'] org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310) at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139) at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310) at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300) at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457) at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922) at org.apache.spark.sql.DataFrame.init(DataFrame.scala:147) at org.apache.spark.sql.DataFrame.init(DataFrame.scala:131) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727) at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57) ``` Author: Cheng Hao hao.ch...@intel.com Author: Cheng Lian l...@databricks.com Author: Yin Huai yh...@databricks.com Closes #6435 from liancheng/classLoader and squashes the following commits: d4c4845 [Cheng Lian] Fixes CliSuite 75e80e2 [Yin Huai] Update the fix. fd26533 [Cheng Hao] scalastyle dd78775 [Cheng Hao] workaround for classloader of IsolatedClientLoader (cherry picked from commit db3fd054f240c7e38aba0732e471df65cd14011a) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e07b7156 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e07b7156 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e07b7156 Branch: refs/heads/branch-1.4 Commit: e07b71560cb791c701ad28adff02f5db6b490136 Parents: 89fe93f Author: Cheng Hao hao.ch...@intel.com Authored: Wed May 27 14:21:00 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 14:31:18 2015 -0700 -- .../spark/sql/hive/thriftserver/CliSuite.scala | 41 ++-- .../org/apache/spark/sql/hive/HiveContext.scala | 18 +++-- .../org/apache/spark/sql/hive/TableReader.scala | 2 +- 3 files changed, 53 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e07b7156/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala -- diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index b070fa8..cc07db8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src
spark git commit: [SPARK-7853] [SQL] Fixes a class loader issue in Spark SQL
Repository: spark Updated Branches: refs/heads/master b97ddff00 - db3fd054f [SPARK-7853] [SQL] Fixes a class loader issue in Spark SQL This PR is based on PR #6396 authored by chenghao-intel. Essentially, Spark SQL should use context classloader to load SerDe classes. yhuai helped updating the test case, and I fixed a bug in the original `CliSuite`: while testing the CLI tool with `runCliWithin`, we don't append `\n` to the last query, thus the last query is never executed. Original PR description is pasted below. ``` bin/spark-sql --jars ./sql/hive/src/test/resources/hive-hcatalog-core-0.13.1.jar CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'; ``` Throws exception like ``` 15/05/26 00:16:33 ERROR SparkSQLDriver: Failed in [CREATE TABLE t1(a string, b string) ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe'] org.apache.spark.sql.execution.QueryExecutionException: FAILED: Execution Error, return code 1 from org.apache.hadoop.hive.ql.exec.DDLTask. Cannot validate serde: org.apache.hive.hcatalog.data.JsonSerDe at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:333) at org.apache.spark.sql.hive.client.ClientWrapper$$anonfun$runHive$1.apply(ClientWrapper.scala:310) at org.apache.spark.sql.hive.client.ClientWrapper.withHiveState(ClientWrapper.scala:139) at org.apache.spark.sql.hive.client.ClientWrapper.runHive(ClientWrapper.scala:310) at org.apache.spark.sql.hive.client.ClientWrapper.runSqlHive(ClientWrapper.scala:300) at org.apache.spark.sql.hive.HiveContext.runSqlHive(HiveContext.scala:457) at org.apache.spark.sql.hive.execution.HiveNativeCommand.run(HiveNativeCommand.scala:33) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult$lzycompute(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.sideEffectResult(commands.scala:57) at org.apache.spark.sql.execution.ExecutedCommand.doExecute(commands.scala:68) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88) at org.apache.spark.sql.execution.SparkPlan$$anonfun$execute$1.apply(SparkPlan.scala:88) at org.apache.spark.rdd.RDDOperationScope$.withScope(RDDOperationScope.scala:148) at org.apache.spark.sql.execution.SparkPlan.execute(SparkPlan.scala:87) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd$lzycompute(SQLContext.scala:922) at org.apache.spark.sql.SQLContext$QueryExecution.toRdd(SQLContext.scala:922) at org.apache.spark.sql.DataFrame.init(DataFrame.scala:147) at org.apache.spark.sql.DataFrame.init(DataFrame.scala:131) at org.apache.spark.sql.DataFrame$.apply(DataFrame.scala:51) at org.apache.spark.sql.SQLContext.sql(SQLContext.scala:727) at org.apache.spark.sql.hive.thriftserver.AbstractSparkSQLDriver.run(AbstractSparkSQLDriver.scala:57) ``` Author: Cheng Hao hao.ch...@intel.com Author: Cheng Lian l...@databricks.com Author: Yin Huai yh...@databricks.com Closes #6435 from liancheng/classLoader and squashes the following commits: d4c4845 [Cheng Lian] Fixes CliSuite 75e80e2 [Yin Huai] Update the fix. fd26533 [Cheng Hao] scalastyle dd78775 [Cheng Hao] workaround for classloader of IsolatedClientLoader Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/db3fd054 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/db3fd054 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/db3fd054 Branch: refs/heads/master Commit: db3fd054f240c7e38aba0732e471df65cd14011a Parents: b97ddff Author: Cheng Hao hao.ch...@intel.com Authored: Wed May 27 14:21:00 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 14:21:00 2015 -0700 -- .../spark/sql/hive/thriftserver/CliSuite.scala | 41 ++-- .../org/apache/spark/sql/hive/HiveContext.scala | 18 +++-- .../org/apache/spark/sql/hive/TableReader.scala | 2 +- 3 files changed, 53 insertions(+), 8 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/db3fd054/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala -- diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index b070fa8..cc07db8 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -25,11 +25,15 @@ import scala.concurrent.{Await
spark git commit: [SPARK-7868] [SQL] Ignores _temporary directories in HadoopFsRelation
Repository: spark Updated Branches: refs/heads/branch-1.4 faadbd4d9 - d0bd68ff8 [SPARK-7868] [SQL] Ignores _temporary directories in HadoopFsRelation So that potential partial/corrupted data files left by failed tasks/jobs won't affect normal data scan. Author: Cheng Lian l...@databricks.com Closes #6411 from liancheng/spark-7868 and squashes the following commits: 273ea36 [Cheng Lian] Ignores _temporary directories (cherry picked from commit b463e6d618e69c535297e51f41eca4f91bd33cc8) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d0bd68ff Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d0bd68ff Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d0bd68ff Branch: refs/heads/branch-1.4 Commit: d0bd68ff8a1dcfbff8e6d40573ca049d208ab2de Parents: faadbd4 Author: Cheng Lian l...@databricks.com Authored: Tue May 26 20:48:56 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue May 26 20:49:05 2015 -0700 -- .../apache/spark/sql/sources/interfaces.scala | 20 +--- .../sql/sources/hadoopFsRelationSuites.scala| 16 2 files changed, 29 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d0bd68ff/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala index aaabbad..c06026e 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/interfaces.scala @@ -31,7 +31,7 @@ import org.apache.spark.SerializableWritable import org.apache.spark.sql.{Row, _} import org.apache.spark.sql.catalyst.expressions._ import org.apache.spark.sql.catalyst.expressions.codegen.GenerateMutableProjection -import org.apache.spark.sql.types.{StructField, StructType} +import org.apache.spark.sql.types.StructType /** * ::DeveloperApi:: @@ -378,16 +378,22 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio var leafDirToChildrenFiles = mutable.Map.empty[Path, Array[FileStatus]] def refresh(): Unit = { + // We don't filter files/directories whose name start with _ or . here, as specific data + // sources may take advantages over them (e.g. Parquet _metadata and _common_metadata files). + // But _temporary directories are explicitly ignored since failed tasks/jobs may leave + // partial/corrupted data files there. def listLeafFilesAndDirs(fs: FileSystem, status: FileStatus): Set[FileStatus] = { -val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDir) -val leafDirs = if (dirs.isEmpty) Set(status) else Set.empty[FileStatus] -files.toSet ++ leafDirs ++ dirs.flatMap(dir = listLeafFilesAndDirs(fs, dir)) +if (status.getPath.getName.toLowerCase == _temporary) { + Set.empty +} else { + val (dirs, files) = fs.listStatus(status.getPath).partition(_.isDir) + val leafDirs = if (dirs.isEmpty) Set(status) else Set.empty[FileStatus] + files.toSet ++ leafDirs ++ dirs.flatMap(dir = listLeafFilesAndDirs(fs, dir)) +} } leafFiles.clear() - // We don't filter files/directories like _temporary/_SUCCESS here, as specific data sources - // may take advantages over them (e.g. Parquet _metadata and _common_metadata files). val statuses = paths.flatMap { path = val hdfsPath = new Path(path) val fs = hdfsPath.getFileSystem(hadoopConf) @@ -395,7 +401,7 @@ abstract class HadoopFsRelation private[sql](maybePartitionSpec: Option[Partitio Try(fs.getFileStatus(qualified)).toOption.toArray.flatMap(listLeafFilesAndDirs(fs, _)) } - val (dirs, files) = statuses.partition(_.isDir) + val files = statuses.filterNot(_.isDir) leafFiles ++= files.map(f = f.getPath - f).toMap leafDirToChildrenFiles ++= files.groupBy(_.getPath.getParent) } http://git-wip-us.apache.org/repos/asf/spark/blob/d0bd68ff/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala index 7c02d56..cf5ae88 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/sources/hadoopFsRelationSuites.scala @@
spark git commit: [SPARK-7950] [SQL] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext()
Repository: spark Updated Branches: refs/heads/master a51b133de - e7b617755 [SPARK-7950] [SQL] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext() When starting `HiveThriftServer2` via `startWithContext`, property `spark.sql.hive.version` isn't set. This causes Simba ODBC driver 1.0.8.1006 behaves differently and fails simple queries. Hive2 JDBC driver works fine in this case. Also, when starting the server with `start-thriftserver.sh`, both Hive2 JDBC driver and Simba ODBC driver works fine. Please refer to [SPARK-7950] [1] for details. [1]: https://issues.apache.org/jira/browse/SPARK-7950 Author: Cheng Lian l...@databricks.com Closes #6500 from liancheng/odbc-bugfix and squashes the following commits: 051e3a3 [Cheng Lian] Fixes import order 3a97376 [Cheng Lian] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext() Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e7b61775 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e7b61775 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e7b61775 Branch: refs/heads/master Commit: e7b61775571ce7a06d044bc3a6055ff94c7477d6 Parents: a51b133 Author: Cheng Lian l...@databricks.com Authored: Fri May 29 10:43:34 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri May 29 10:43:34 2015 -0700 -- .../sql/hive/thriftserver/HiveThriftServer2.scala| 15 --- 1 file changed, 8 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e7b61775/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 3458b04..94687ee 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -17,23 +17,23 @@ package org.apache.spark.sql.hive.thriftserver +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} -import org.apache.spark.sql.SQLConf -import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListenerApplicationEnd, SparkListener} import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab +import org.apache.spark.sql.hive.{HiveContext, HiveShim} import org.apache.spark.util.Utils - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{Logging, SparkContext} /** * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a @@ -51,6 +51,7 @@ object HiveThriftServer2 extends Logging { @DeveloperApi def startWithContext(sqlContext: HiveContext): Unit = { val server = new HiveThriftServer2(sqlContext) +sqlContext.setConf(spark.sql.hive.version, HiveShim.version) server.init(sqlContext.hiveconf) server.start() listener = new HiveThriftServer2Listener(server, sqlContext.conf) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7950] [SQL] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext()
Repository: spark Updated Branches: refs/heads/branch-1.4 23bd05fff - caea7a618 [SPARK-7950] [SQL] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext() When starting `HiveThriftServer2` via `startWithContext`, property `spark.sql.hive.version` isn't set. This causes Simba ODBC driver 1.0.8.1006 behaves differently and fails simple queries. Hive2 JDBC driver works fine in this case. Also, when starting the server with `start-thriftserver.sh`, both Hive2 JDBC driver and Simba ODBC driver works fine. Please refer to [SPARK-7950] [1] for details. [1]: https://issues.apache.org/jira/browse/SPARK-7950 Author: Cheng Lian l...@databricks.com Closes #6500 from liancheng/odbc-bugfix and squashes the following commits: 051e3a3 [Cheng Lian] Fixes import order 3a97376 [Cheng Lian] Sets spark.sql.hive.version in HiveThriftServer2.startWithContext() (cherry picked from commit e7b61775571ce7a06d044bc3a6055ff94c7477d6) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/caea7a61 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/caea7a61 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/caea7a61 Branch: refs/heads/branch-1.4 Commit: caea7a618db7989a37ee59fcf928678efadba3e0 Parents: 23bd05f Author: Cheng Lian l...@databricks.com Authored: Fri May 29 10:43:34 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri May 29 10:43:44 2015 -0700 -- .../sql/hive/thriftserver/HiveThriftServer2.scala| 15 --- 1 file changed, 8 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/caea7a61/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala index 3458b04..94687ee 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/HiveThriftServer2.scala @@ -17,23 +17,23 @@ package org.apache.spark.sql.hive.thriftserver +import scala.collection.mutable +import scala.collection.mutable.ArrayBuffer + import org.apache.commons.logging.LogFactory import org.apache.hadoop.hive.conf.HiveConf import org.apache.hadoop.hive.conf.HiveConf.ConfVars import org.apache.hive.service.cli.thrift.{ThriftBinaryCLIService, ThriftHttpCLIService} import org.apache.hive.service.server.{HiveServer2, ServerOptionsProcessor} -import org.apache.spark.sql.SQLConf -import org.apache.spark.{SparkContext, SparkConf, Logging} import org.apache.spark.annotation.DeveloperApi -import org.apache.spark.sql.hive.HiveContext +import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd, SparkListenerJobStart} +import org.apache.spark.sql.SQLConf import org.apache.spark.sql.hive.thriftserver.ReflectionUtils._ -import org.apache.spark.scheduler.{SparkListenerJobStart, SparkListenerApplicationEnd, SparkListener} import org.apache.spark.sql.hive.thriftserver.ui.ThriftServerTab +import org.apache.spark.sql.hive.{HiveContext, HiveShim} import org.apache.spark.util.Utils - -import scala.collection.mutable -import scala.collection.mutable.ArrayBuffer +import org.apache.spark.{Logging, SparkContext} /** * The main entry point for the Spark SQL port of HiveServer2. Starts up a `SparkSQLContext` and a @@ -51,6 +51,7 @@ object HiveThriftServer2 extends Logging { @DeveloperApi def startWithContext(sqlContext: HiveContext): Unit = { val server = new HiveThriftServer2(sqlContext) +sqlContext.setConf(spark.sql.hive.version, HiveShim.version) server.init(sqlContext.hiveconf) server.start() listener = new HiveThriftServer2Listener(server, sqlContext.conf) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7907] [SQL] [UI] Rename tab ThriftServer to SQL.
Repository: spark Updated Branches: refs/heads/branch-1.4 b4ecbce65 - bd9173c14 [SPARK-7907] [SQL] [UI] Rename tab ThriftServer to SQL. This PR has three changes: 1. Renaming the table of `ThriftServer` to `SQL`; 2. Renaming the title of the tab from `ThriftServer` to `JDBC/ODBC Server`; and 3. Renaming the title of the session page from `ThriftServer` to `JDBC/ODBC Session`. https://issues.apache.org/jira/browse/SPARK-7907 Author: Yin Huai yh...@databricks.com Closes #6448 from yhuai/JDBCServer and squashes the following commits: eadcc3d [Yin Huai] Update test. 9168005 [Yin Huai] Use SQL as the tab name. 221831e [Yin Huai] Rename ThriftServer to JDBCServer. (cherry picked from commit 3c1f1baaf003d50786d3eee1e288f4bac69096f2) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bd9173c1 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bd9173c1 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bd9173c1 Branch: refs/heads/branch-1.4 Commit: bd9173c14c4a25b6f87797eae348634e7aa7f7ac Parents: b4ecbce Author: Yin Huai yh...@databricks.com Authored: Wed May 27 20:04:29 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 20:04:57 2015 -0700 -- .../apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala | 4 ++-- .../spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 2 +- .../apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala | 4 +++- .../org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala | 4 ++-- 4 files changed, 8 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bd9173c1/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 6a2be4a..7c48ff4 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -47,7 +47,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage( /h4 ++ generateSessionStatsTable() ++ generateSQLStatsTable() -UIUtils.headerSparkPage(ThriftServer, content, parent, Some(5000)) +UIUtils.headerSparkPage(JDBC/ODBC Server, content, parent, Some(5000)) } /** Generate basic stats of the thrift server program */ @@ -143,7 +143,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage( val headerRow = Seq(User, IP, Session ID, Start Time, Finish Time, Duration, Total Execute) def generateDataRow(session: SessionInfo): Seq[Node] = { -val sessionLink = %s/ThriftServer/session?id=%s +val sessionLink = %s/sql/session?id=%s .format(UIUtils.prependBaseUri(parent.basePath), session.sessionId) tr td {session.userName} /td http://git-wip-us.apache.org/repos/asf/spark/blob/bd9173c1/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 33ba038..d9d66dc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -55,7 +55,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) Total run {sessionStat._2.totalExecution} SQL /h4 ++ generateSQLStatsTable(sessionStat._2.sessionId) -UIUtils.headerSparkPage(ThriftServer, content, parent, Some(5000)) +UIUtils.headerSparkPage(JDBC/ODBC Session, content, parent, Some(5000)) } /** Generate basic stats of the streaming program */ http://git-wip-us.apache.org/repos/asf/spark/blob/bd9173c1/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui
spark git commit: [SPARK-7907] [SQL] [UI] Rename tab ThriftServer to SQL.
Repository: spark Updated Branches: refs/heads/master a1e092eae - 3c1f1baaf [SPARK-7907] [SQL] [UI] Rename tab ThriftServer to SQL. This PR has three changes: 1. Renaming the table of `ThriftServer` to `SQL`; 2. Renaming the title of the tab from `ThriftServer` to `JDBC/ODBC Server`; and 3. Renaming the title of the session page from `ThriftServer` to `JDBC/ODBC Session`. https://issues.apache.org/jira/browse/SPARK-7907 Author: Yin Huai yh...@databricks.com Closes #6448 from yhuai/JDBCServer and squashes the following commits: eadcc3d [Yin Huai] Update test. 9168005 [Yin Huai] Use SQL as the tab name. 221831e [Yin Huai] Rename ThriftServer to JDBCServer. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3c1f1baa Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3c1f1baa Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3c1f1baa Branch: refs/heads/master Commit: 3c1f1baaf003d50786d3eee1e288f4bac69096f2 Parents: a1e092e Author: Yin Huai yh...@databricks.com Authored: Wed May 27 20:04:29 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Wed May 27 20:04:29 2015 -0700 -- .../apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala | 4 ++-- .../spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala | 2 +- .../apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala | 4 +++- .../org/apache/spark/sql/hive/thriftserver/UISeleniumSuite.scala | 4 ++-- 4 files changed, 8 insertions(+), 6 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3c1f1baa/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala index 6a2be4a..7c48ff4 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerPage.scala @@ -47,7 +47,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage( /h4 ++ generateSessionStatsTable() ++ generateSQLStatsTable() -UIUtils.headerSparkPage(ThriftServer, content, parent, Some(5000)) +UIUtils.headerSparkPage(JDBC/ODBC Server, content, parent, Some(5000)) } /** Generate basic stats of the thrift server program */ @@ -143,7 +143,7 @@ private[ui] class ThriftServerPage(parent: ThriftServerTab) extends WebUIPage( val headerRow = Seq(User, IP, Session ID, Start Time, Finish Time, Duration, Total Execute) def generateDataRow(session: SessionInfo): Seq[Node] = { -val sessionLink = %s/ThriftServer/session?id=%s +val sessionLink = %s/sql/session?id=%s .format(UIUtils.prependBaseUri(parent.basePath), session.sessionId) tr td {session.userName} /td http://git-wip-us.apache.org/repos/asf/spark/blob/3c1f1baa/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala index 33ba038..d9d66dc 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerSessionPage.scala @@ -55,7 +55,7 @@ private[ui] class ThriftServerSessionPage(parent: ThriftServerTab) Total run {sessionStat._2.totalExecution} SQL /h4 ++ generateSQLStatsTable(sessionStat._2.sessionId) -UIUtils.headerSparkPage(ThriftServer, content, parent, Some(5000)) +UIUtils.headerSparkPage(JDBC/ODBC Session, content, parent, Some(5000)) } /** Generate basic stats of the streaming program */ http://git-wip-us.apache.org/repos/asf/spark/blob/3c1f1baa/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/ui/ThriftServerTab.scala index 343031f..94fd8a6 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive
spark git commit: [HOT-FIX] Add EvaluatedType back to RDG
Repository: spark Updated Branches: refs/heads/branch-1.4 97fedf1a0 - 8c3fc3a6c [HOT-FIX] Add EvaluatedType back to RDG https://github.com/apache/spark/commit/87941ff8c49a6661f22c31aa7b84ac1fce768135 accidentally removed the EvaluatedType. Author: Yin Huai yh...@databricks.com Closes #6589 from yhuai/getBackEvaluatedType and squashes the following commits: 618c2eb [Yin Huai] Add EvaluatedType back. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8c3fc3a6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8c3fc3a6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8c3fc3a6 Branch: refs/heads/branch-1.4 Commit: 8c3fc3a6cd3fe05b018733c68691837de3b06f4a Parents: 97fedf1 Author: Yin Huai yh...@databricks.com Authored: Tue Jun 2 09:59:19 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue Jun 2 09:59:19 2015 -0700 -- .../scala/org/apache/spark/sql/catalyst/expressions/random.scala | 2 ++ 1 file changed, 2 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8c3fc3a6/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala index b264712..5677037 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/random.scala @@ -38,6 +38,8 @@ abstract class RDG(seed: Long) extends LeafExpression with Serializable { */ @transient protected lazy val rng = new XORShiftRandom(seed + TaskContext.get().partitionId()) + override type EvaluatedType = Double + override def deterministic: Boolean = false override def nullable: Boolean = false - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8776] Increase the default MaxPermSize
Repository: spark Updated Branches: refs/heads/branch-1.4 de0802499 - f142867ec [SPARK-8776] Increase the default MaxPermSize I am increasing the perm gen size to 256m. https://issues.apache.org/jira/browse/SPARK-8776 Author: Yin Huai yh...@databricks.com Closes #7196 from yhuai/SPARK-8776 and squashes the following commits: 60901b4 [Yin Huai] Fix test. d44b713 [Yin Huai] Make sparkShell and hiveConsole use 256m PermGen size. 30aaf8e [Yin Huai] Increase the default PermGen size to 256m. (cherry picked from commit f743c79abe5a2fb66be32a896ea47e858569b0c7) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f142867e Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f142867e Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f142867e Branch: refs/heads/branch-1.4 Commit: f142867ecee59a635df91aee888351bee5f29c0e Parents: de08024 Author: Yin Huai yh...@databricks.com Authored: Thu Jul 2 22:09:07 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu Jul 2 22:09:33 2015 -0700 -- .../java/org/apache/spark/launcher/AbstractCommandBuilder.java | 2 +- .../apache/spark/launcher/SparkSubmitCommandBuilderSuite.java| 2 +- project/SparkBuild.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f142867e/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java -- diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index b8f02b9..b2591c2 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -133,7 +133,7 @@ abstract class AbstractCommandBuilder { } } -cmd.add(-XX:MaxPermSize=128m); +cmd.add(-XX:MaxPermSize=256m); } void addOptionString(ListString cmd, String options) { http://git-wip-us.apache.org/repos/asf/spark/blob/f142867e/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java -- diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 97043a7..7329ac9 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -194,7 +194,7 @@ public class SparkSubmitCommandBuilderSuite { if (isDriver) { assertEquals(-XX:MaxPermSize=256m, arg); } else { - assertEquals(-XX:MaxPermSize=128m, arg); + assertEquals(-XX:MaxPermSize=256m, arg); } } } http://git-wip-us.apache.org/repos/asf/spark/blob/f142867e/project/SparkBuild.scala -- diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index aa59d74..4b00a66 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -206,7 +206,7 @@ object SparkBuild extends PomBuild { fork := true, outputStrategy in run := Some (StdoutOutput), -javaOptions ++= Seq(-Xmx2G, -XX:MaxPermSize=1g), +javaOptions ++= Seq(-Xmx2G, -XX:MaxPermSize=256m), sparkShell := { (runMain in Compile).toTask( org.apache.spark.repl.Main -usejavacp).value @@ -307,7 +307,7 @@ object SQL { object Hive { lazy val settings = Seq( -javaOptions += -XX:MaxPermSize=1g, +javaOptions += -XX:MaxPermSize=256m, // Specially disable assertions since some Hive tests fail them javaOptions in Test := (javaOptions in Test).value.filterNot(_ == -ea), // Multiple queries rely on the TestHive singleton. See comments there for more details. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8776] Increase the default MaxPermSize
Repository: spark Updated Branches: refs/heads/master a59d14f62 - f743c79ab [SPARK-8776] Increase the default MaxPermSize I am increasing the perm gen size to 256m. https://issues.apache.org/jira/browse/SPARK-8776 Author: Yin Huai yh...@databricks.com Closes #7196 from yhuai/SPARK-8776 and squashes the following commits: 60901b4 [Yin Huai] Fix test. d44b713 [Yin Huai] Make sparkShell and hiveConsole use 256m PermGen size. 30aaf8e [Yin Huai] Increase the default PermGen size to 256m. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f743c79a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f743c79a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f743c79a Branch: refs/heads/master Commit: f743c79abe5a2fb66be32a896ea47e858569b0c7 Parents: a59d14f Author: Yin Huai yh...@databricks.com Authored: Thu Jul 2 22:09:07 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu Jul 2 22:09:07 2015 -0700 -- .../java/org/apache/spark/launcher/AbstractCommandBuilder.java | 2 +- .../apache/spark/launcher/SparkSubmitCommandBuilderSuite.java| 2 +- project/SparkBuild.scala | 4 ++-- 3 files changed, 4 insertions(+), 4 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f743c79a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java -- diff --git a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java index 33d65d1..5e793a5 100644 --- a/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java +++ b/launcher/src/main/java/org/apache/spark/launcher/AbstractCommandBuilder.java @@ -136,7 +136,7 @@ abstract class AbstractCommandBuilder { } } -cmd.add(-XX:MaxPermSize=128m); +cmd.add(-XX:MaxPermSize=256m); } void addOptionString(ListString cmd, String options) { http://git-wip-us.apache.org/repos/asf/spark/blob/f743c79a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java -- diff --git a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java index 97043a7..7329ac9 100644 --- a/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java +++ b/launcher/src/test/java/org/apache/spark/launcher/SparkSubmitCommandBuilderSuite.java @@ -194,7 +194,7 @@ public class SparkSubmitCommandBuilderSuite { if (isDriver) { assertEquals(-XX:MaxPermSize=256m, arg); } else { - assertEquals(-XX:MaxPermSize=128m, arg); + assertEquals(-XX:MaxPermSize=256m, arg); } } } http://git-wip-us.apache.org/repos/asf/spark/blob/f743c79a/project/SparkBuild.scala -- diff --git a/project/SparkBuild.scala b/project/SparkBuild.scala index 5f389bc..3408c6d 100644 --- a/project/SparkBuild.scala +++ b/project/SparkBuild.scala @@ -206,7 +206,7 @@ object SparkBuild extends PomBuild { fork := true, outputStrategy in run := Some (StdoutOutput), -javaOptions ++= Seq(-Xmx2G, -XX:MaxPermSize=1g), +javaOptions ++= Seq(-Xmx2G, -XX:MaxPermSize=256m), sparkShell := { (runMain in Compile).toTask( org.apache.spark.repl.Main -usejavacp).value @@ -299,7 +299,7 @@ object SQL { object Hive { lazy val settings = Seq( -javaOptions += -XX:MaxPermSize=1g, +javaOptions += -XX:MaxPermSize=256m, // Specially disable assertions since some Hive tests fail them javaOptions in Test := (javaOptions in Test).value.filterNot(_ == -ea), // Multiple queries rely on the TestHive singleton. See comments there for more details. - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7805] [SQL] Move SQLTestUtils.scala and ParquetTest.scala to src/test
Repository: spark Updated Branches: refs/heads/master bfbc0df72 - ed21476bc [SPARK-7805] [SQL] Move SQLTestUtils.scala and ParquetTest.scala to src/test https://issues.apache.org/jira/browse/SPARK-7805 Because `sql/hive`'s tests depend on the test jar of `sql/core`, we do not need to store `SQLTestUtils` and `ParquetTest` in `src/main`. We should only add stuff that will be needed by `sql/console` or Python tests (for Python, we need it in `src/main`, right? davies). Author: Yin Huai yh...@databricks.com Closes #6334 from yhuai/SPARK-7805 and squashes the following commits: af6d0c9 [Yin Huai] mima b86746a [Yin Huai] Move SQLTestUtils.scala and ParquetTest.scala to src/test. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ed21476b Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ed21476b Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ed21476b Branch: refs/heads/master Commit: ed21476bc0c760616e7e6bb99f6541745fb09595 Parents: bfbc0df Author: Yin Huai yh...@databricks.com Authored: Sun May 24 09:51:37 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun May 24 09:51:37 2015 -0700 -- project/MimaExcludes.scala | 5 +- .../apache/spark/sql/parquet/ParquetTest.scala | 102 --- .../scala/org/apache/spark/sql/test/README.md | 7 ++ .../apache/spark/sql/test/SQLTestUtils.scala| 88 .../apache/spark/sql/parquet/ParquetTest.scala | 102 +++ .../apache/spark/sql/test/SQLTestUtils.scala| 88 6 files changed, 201 insertions(+), 191 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ed21476b/project/MimaExcludes.scala -- diff --git a/project/MimaExcludes.scala b/project/MimaExcludes.scala index 03e93a2..11b439e 100644 --- a/project/MimaExcludes.scala +++ b/project/MimaExcludes.scala @@ -133,7 +133,10 @@ object MimaExcludes { org.apache.spark.sql.parquet.TestGroupWriteSupport), ProblemFilters.exclude[MissingClassProblem](org.apache.spark.sql.CachedData), ProblemFilters.exclude[MissingClassProblem](org.apache.spark.sql.CachedData$), - ProblemFilters.exclude[MissingClassProblem](org.apache.spark.sql.CacheManager) + ProblemFilters.exclude[MissingClassProblem](org.apache.spark.sql.CacheManager), +// TODO: Remove the following rule once ParquetTest has been moved to src/test. +ProblemFilters.exclude[MissingClassProblem]( + org.apache.spark.sql.parquet.ParquetTest) ) ++ Seq( // SPARK-7530 Added StreamingContext.getState() ProblemFilters.exclude[MissingMethodProblem]( http://git-wip-us.apache.org/repos/asf/spark/blob/ed21476b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala deleted file mode 100644 index 516ba37..000 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/ParquetTest.scala +++ /dev/null @@ -1,102 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the License); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - *http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an AS IS BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.spark.sql.parquet - -import java.io.File - -import scala.reflect.ClassTag -import scala.reflect.runtime.universe.TypeTag - -import org.apache.spark.sql.test.SQLTestUtils -import org.apache.spark.sql.{DataFrame, SaveMode} - -/** - * A helper trait that provides convenient facilities for Parquet testing. - * - * NOTE: Considering classes `Tuple1` ... `Tuple22` all extend `Product`, it would be more - * convenient to use tuples rather than special case classes when writing test cases/suites. - * Especially, `Tuple1.apply` can be used to easily wrap a single type/value. - */ -private[sql] trait ParquetTest extends SQLTestUtils { - import sqlContext.implicits
spark git commit: [SPARK-7845] [BUILD] Bump Hadoop 1 tests to version 1.2.1
Repository: spark Updated Branches: refs/heads/branch-1.4 947d700ec - 11d998eb7 [SPARK-7845] [BUILD] Bump Hadoop 1 tests to version 1.2.1 https://issues.apache.org/jira/browse/SPARK-7845 Author: Yin Huai yh...@databricks.com Closes #6384 from yhuai/hadoop1Test and squashes the following commits: 82fcea8 [Yin Huai] Use hadoop 1.2.1 (a stable version) for hadoop 1 test. (cherry picked from commit bfbc0df72944fe0a900ab920d8c4329a11fccca6) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/11d998eb Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/11d998eb Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/11d998eb Branch: refs/heads/branch-1.4 Commit: 11d998eb774d93797155d508a0e2bfb432da145f Parents: 947d700 Author: Yin Huai yh...@databricks.com Authored: Sun May 24 09:49:57 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun May 24 09:50:12 2015 -0700 -- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/11d998eb/dev/run-tests -- diff --git a/dev/run-tests b/dev/run-tests index 65a5cb9..d5d2a42 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -40,7 +40,7 @@ function handle_error () { { if [ -n $AMPLAB_JENKINS_BUILD_PROFILE ]; then if [ $AMPLAB_JENKINS_BUILD_PROFILE = hadoop1.0 ]; then - export SBT_MAVEN_PROFILES_ARGS=-Phadoop-1 -Dhadoop.version=1.0.4 + export SBT_MAVEN_PROFILES_ARGS=-Phadoop-1 -Dhadoop.version=1.2.1 elif [ $AMPLAB_JENKINS_BUILD_PROFILE = hadoop2.0 ]; then export SBT_MAVEN_PROFILES_ARGS=-Phadoop-1 -Dhadoop.version=2.0.0-mr1-cdh4.1.1 elif [ $AMPLAB_JENKINS_BUILD_PROFILE = hadoop2.2 ]; then - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7845] [BUILD] Bump Hadoop 1 tests to version 1.2.1
Repository: spark Updated Branches: refs/heads/master 3c1a2d049 - bfbc0df72 [SPARK-7845] [BUILD] Bump Hadoop 1 tests to version 1.2.1 https://issues.apache.org/jira/browse/SPARK-7845 Author: Yin Huai yh...@databricks.com Closes #6384 from yhuai/hadoop1Test and squashes the following commits: 82fcea8 [Yin Huai] Use hadoop 1.2.1 (a stable version) for hadoop 1 test. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/bfbc0df7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/bfbc0df7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/bfbc0df7 Branch: refs/heads/master Commit: bfbc0df72944fe0a900ab920d8c4329a11fccca6 Parents: 3c1a2d0 Author: Yin Huai yh...@databricks.com Authored: Sun May 24 09:49:57 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun May 24 09:49:57 2015 -0700 -- dev/run-tests | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/bfbc0df7/dev/run-tests -- diff --git a/dev/run-tests b/dev/run-tests index 44d8027..57296d0 100755 --- a/dev/run-tests +++ b/dev/run-tests @@ -40,7 +40,7 @@ function handle_error () { { if [ -n $AMPLAB_JENKINS_BUILD_PROFILE ]; then if [ $AMPLAB_JENKINS_BUILD_PROFILE = hadoop1.0 ]; then - export SBT_MAVEN_PROFILES_ARGS=-Phadoop-1 -Dhadoop.version=1.0.4 + export SBT_MAVEN_PROFILES_ARGS=-Phadoop-1 -Dhadoop.version=1.2.1 elif [ $AMPLAB_JENKINS_BUILD_PROFILE = hadoop2.0 ]; then export SBT_MAVEN_PROFILES_ARGS=-Phadoop-1 -Dhadoop.version=2.0.0-mr1-cdh4.1.1 elif [ $AMPLAB_JENKINS_BUILD_PROFILE = hadoop2.2 ]; then - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7322, SPARK-7836, SPARK-7822][SQL] DataFrame window function related updates
Repository: spark Updated Branches: refs/heads/master ad0badba1 - efe3bfdf4 [SPARK-7322, SPARK-7836, SPARK-7822][SQL] DataFrame window function related updates 1. ntile should take an integer as parameter. 2. Added Python API (based on #6364) 3. Update documentation of various DataFrame Python functions. Author: Davies Liu dav...@databricks.com Author: Reynold Xin r...@databricks.com Closes #6374 from rxin/window-final and squashes the following commits: 69004c7 [Reynold Xin] Style fix. 288cea9 [Reynold Xin] Update documentaiton. 7cb8985 [Reynold Xin] Merge pull request #6364 from davies/window 66092b4 [Davies Liu] update docs ed73cb4 [Reynold Xin] [SPARK-7322][SQL] Improve DataFrame window function documentation. ef55132 [Davies Liu] Merge branch 'master' of github.com:apache/spark into window4 8936ade [Davies Liu] fix maxint in python 3 2649358 [Davies Liu] update docs 778e2c0 [Davies Liu] SPARK-7836 and SPARK-7822: Python API of window functions Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/efe3bfdf Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/efe3bfdf Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/efe3bfdf Branch: refs/heads/master Commit: efe3bfdf496aa6206ace2697e31dd4c0c3c824fb Parents: ad0badb Author: Davies Liu dav...@databricks.com Authored: Sat May 23 08:30:05 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sat May 23 08:30:05 2015 -0700 -- python/pyspark/sql/__init__.py | 25 +-- python/pyspark/sql/column.py| 54 +++-- python/pyspark/sql/context.py | 2 - python/pyspark/sql/dataframe.py | 2 + python/pyspark/sql/functions.py | 147 +++--- python/pyspark/sql/group.py | 2 + python/pyspark/sql/tests.py | 31 ++- python/pyspark/sql/window.py| 158 +++ .../scala/org/apache/spark/sql/functions.scala | 197 +-- .../sql/hive/HiveDataFrameWindowSuite.scala | 20 +- 10 files changed, 464 insertions(+), 174 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/efe3bfdf/python/pyspark/sql/__init__.py -- diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 66b0bff..8fee92a 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -18,26 +18,28 @@ Important classes of Spark SQL and DataFrames: -- L{SQLContext} +- :class:`pyspark.sql.SQLContext` Main entry point for :class:`DataFrame` and SQL functionality. -- L{DataFrame} +- :class:`pyspark.sql.DataFrame` A distributed collection of data grouped into named columns. -- L{Column} +- :class:`pyspark.sql.Column` A column expression in a :class:`DataFrame`. -- L{Row} +- :class:`pyspark.sql.Row` A row of data in a :class:`DataFrame`. -- L{HiveContext} +- :class:`pyspark.sql.HiveContext` Main entry point for accessing data stored in Apache Hive. -- L{GroupedData} +- :class:`pyspark.sql.GroupedData` Aggregation methods, returned by :func:`DataFrame.groupBy`. -- L{DataFrameNaFunctions} +- :class:`pyspark.sql.DataFrameNaFunctions` Methods for handling missing data (null values). -- L{DataFrameStatFunctions} +- :class:`pyspark.sql.DataFrameStatFunctions` Methods for statistics functionality. -- L{functions} +- :class:`pyspark.sql.functions` List of built-in functions available for :class:`DataFrame`. -- L{types} +- :class:`pyspark.sql.types` List of data types available. +- :class:`pyspark.sql.Window` + For working with window functions. from __future__ import absolute_import @@ -66,8 +68,9 @@ from pyspark.sql.column import Column from pyspark.sql.dataframe import DataFrame, SchemaRDD, DataFrameNaFunctions, DataFrameStatFunctions from pyspark.sql.group import GroupedData from pyspark.sql.readwriter import DataFrameReader, DataFrameWriter +from pyspark.sql.window import Window, WindowSpec __all__ = [ 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', -'DataFrameNaFunctions', 'DataFrameStatFunctions' +'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec', ] http://git-wip-us.apache.org/repos/asf/spark/blob/efe3bfdf/python/pyspark/sql/column.py -- diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index baf1ecb..8dc5039 100644 --- a/python/pyspark/sql/column.py +++ b/python/pyspark/sql/column.py @@ -116,6 +116,8 @@ class Column(object): df.colName + 1
spark git commit: [SPARK-7322, SPARK-7836, SPARK-7822][SQL] DataFrame window function related updates
Repository: spark Updated Branches: refs/heads/branch-1.4 ea9db50bc - d1515381c [SPARK-7322, SPARK-7836, SPARK-7822][SQL] DataFrame window function related updates 1. ntile should take an integer as parameter. 2. Added Python API (based on #6364) 3. Update documentation of various DataFrame Python functions. Author: Davies Liu dav...@databricks.com Author: Reynold Xin r...@databricks.com Closes #6374 from rxin/window-final and squashes the following commits: 69004c7 [Reynold Xin] Style fix. 288cea9 [Reynold Xin] Update documentaiton. 7cb8985 [Reynold Xin] Merge pull request #6364 from davies/window 66092b4 [Davies Liu] update docs ed73cb4 [Reynold Xin] [SPARK-7322][SQL] Improve DataFrame window function documentation. ef55132 [Davies Liu] Merge branch 'master' of github.com:apache/spark into window4 8936ade [Davies Liu] fix maxint in python 3 2649358 [Davies Liu] update docs 778e2c0 [Davies Liu] SPARK-7836 and SPARK-7822: Python API of window functions (cherry picked from commit efe3bfdf496aa6206ace2697e31dd4c0c3c824fb) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/d1515381 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/d1515381 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/d1515381 Branch: refs/heads/branch-1.4 Commit: d1515381cb957f40daf026144ce3ac014660df23 Parents: ea9db50 Author: Davies Liu dav...@databricks.com Authored: Sat May 23 08:30:05 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sat May 23 08:30:18 2015 -0700 -- python/pyspark/sql/__init__.py | 25 +-- python/pyspark/sql/column.py| 54 +++-- python/pyspark/sql/context.py | 2 - python/pyspark/sql/dataframe.py | 2 + python/pyspark/sql/functions.py | 147 +++--- python/pyspark/sql/group.py | 2 + python/pyspark/sql/tests.py | 31 ++- python/pyspark/sql/window.py| 158 +++ .../scala/org/apache/spark/sql/functions.scala | 197 +-- .../sql/hive/HiveDataFrameWindowSuite.scala | 20 +- 10 files changed, 464 insertions(+), 174 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/d1515381/python/pyspark/sql/__init__.py -- diff --git a/python/pyspark/sql/__init__.py b/python/pyspark/sql/__init__.py index 66b0bff..8fee92a 100644 --- a/python/pyspark/sql/__init__.py +++ b/python/pyspark/sql/__init__.py @@ -18,26 +18,28 @@ Important classes of Spark SQL and DataFrames: -- L{SQLContext} +- :class:`pyspark.sql.SQLContext` Main entry point for :class:`DataFrame` and SQL functionality. -- L{DataFrame} +- :class:`pyspark.sql.DataFrame` A distributed collection of data grouped into named columns. -- L{Column} +- :class:`pyspark.sql.Column` A column expression in a :class:`DataFrame`. -- L{Row} +- :class:`pyspark.sql.Row` A row of data in a :class:`DataFrame`. -- L{HiveContext} +- :class:`pyspark.sql.HiveContext` Main entry point for accessing data stored in Apache Hive. -- L{GroupedData} +- :class:`pyspark.sql.GroupedData` Aggregation methods, returned by :func:`DataFrame.groupBy`. -- L{DataFrameNaFunctions} +- :class:`pyspark.sql.DataFrameNaFunctions` Methods for handling missing data (null values). -- L{DataFrameStatFunctions} +- :class:`pyspark.sql.DataFrameStatFunctions` Methods for statistics functionality. -- L{functions} +- :class:`pyspark.sql.functions` List of built-in functions available for :class:`DataFrame`. -- L{types} +- :class:`pyspark.sql.types` List of data types available. +- :class:`pyspark.sql.Window` + For working with window functions. from __future__ import absolute_import @@ -66,8 +68,9 @@ from pyspark.sql.column import Column from pyspark.sql.dataframe import DataFrame, SchemaRDD, DataFrameNaFunctions, DataFrameStatFunctions from pyspark.sql.group import GroupedData from pyspark.sql.readwriter import DataFrameReader, DataFrameWriter +from pyspark.sql.window import Window, WindowSpec __all__ = [ 'SQLContext', 'HiveContext', 'DataFrame', 'GroupedData', 'Column', 'Row', -'DataFrameNaFunctions', 'DataFrameStatFunctions' +'DataFrameNaFunctions', 'DataFrameStatFunctions', 'Window', 'WindowSpec', ] http://git-wip-us.apache.org/repos/asf/spark/blob/d1515381/python/pyspark/sql/column.py -- diff --git a/python/pyspark/sql/column.py b/python/pyspark/sql/column.py index baf1ecb..8dc5039 100644 ---
spark git commit: [SPARK-7654] [SQL] Move insertInto into reader/writer interface.
Repository: spark Updated Branches: refs/heads/master a4df0f2d8 - 2b7e63585 [SPARK-7654] [SQL] Move insertInto into reader/writer interface. This one continues the work of https://github.com/apache/spark/pull/6216. Author: Yin Huai yh...@databricks.com Author: Reynold Xin r...@databricks.com Closes #6366 from yhuai/insert and squashes the following commits: 3d717fb [Yin Huai] Use insertInto to handle the casue when table exists and Append is used for saveAsTable. 56d2540 [Yin Huai] Add PreWriteCheck to HiveContext's analyzer. c636e35 [Yin Huai] Remove unnecessary empty lines. cf83837 [Yin Huai] Move insertInto to write. Also, remove the partition columns from InsertIntoHadoopFsRelation. 0841a54 [Reynold Xin] Removed experimental tag for deprecated methods. 33ed8ef [Reynold Xin] [SPARK-7654][SQL] Move insertInto into reader/writer interface. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/2b7e6358 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/2b7e6358 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/2b7e6358 Branch: refs/heads/master Commit: 2b7e63585d61be2dab78b70af3867cda3983d5b1 Parents: a4df0f2 Author: Yin Huai yh...@databricks.com Authored: Sat May 23 09:48:20 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sat May 23 09:48:20 2015 -0700 -- .../scala/org/apache/spark/sql/DataFrame.scala | 52 +++ .../org/apache/spark/sql/DataFrameReader.scala | 18 +- .../org/apache/spark/sql/DataFrameWriter.scala | 66 +--- .../spark/sql/parquet/ParquetTableSupport.scala | 2 +- .../spark/sql/sources/DataSourceStrategy.scala | 5 +- .../org/apache/spark/sql/sources/commands.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 1 - .../org/apache/spark/sql/sources/rules.scala| 19 +- .../org/apache/spark/sql/hive/HiveContext.scala | 4 ++ .../sql/hive/InsertIntoHiveTableSuite.scala | 6 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 8 +-- .../sql/hive/execution/SQLQuerySuite.scala | 8 +-- .../apache/spark/sql/hive/parquetSuites.scala | 4 +- .../sql/sources/hadoopFsRelationSuites.scala| 10 --- 14 files changed, 116 insertions(+), 89 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/2b7e6358/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 3ec1c4a..f968577 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1395,28 +1395,6 @@ class DataFrame private[sql]( def write: DataFrameWriter = new DataFrameWriter(this) /** - * :: Experimental :: - * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. - * @group output - * @since 1.3.0 - */ - @Experimental - def insertInto(tableName: String, overwrite: Boolean): Unit = { -sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd - } - - /** - * :: Experimental :: - * Adds the rows from this RDD to the specified table. - * Throws an exception if the table already exists. - * @group output - * @since 1.3.0 - */ - @Experimental - def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false) - - /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. * @group rdd * @since 1.3.0 @@ -1551,13 +1529,7 @@ class DataFrame private[sql]( */ @deprecated(Use write.mode(mode).saveAsTable(tableName), 1.4.0) def saveAsTable(tableName: String, mode: SaveMode): Unit = { -if (sqlContext.catalog.tableExists(Seq(tableName)) mode == SaveMode.Append) { - // If table already exists and the save mode is Append, - // we will just call insertInto to append the contents of this DataFrame. - insertInto(tableName, overwrite = false) -} else { - write.mode(mode).saveAsTable(tableName) -} +write.mode(mode).saveAsTable(tableName) } /** @@ -1713,9 +1685,29 @@ class DataFrame private[sql]( write.format(source).mode(mode).options(options).save() } + + /** + * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. + * @group output + */ + @deprecated(Use write.mode(SaveMode.Append|SaveMode.Overwrite).saveAsTable(tableName), 1.4.0) + def insertInto(tableName: String, overwrite: Boolean): Unit = { +write.mode(if (overwrite) SaveMode.Overwrite else SaveMode.Append).insertInto(tableName) + } + + /** + * Adds
spark git commit: [SPARK-7654] [SQL] Move insertInto into reader/writer interface.
Repository: spark Updated Branches: refs/heads/branch-1.4 c6e574213 - 8d6d8a538 [SPARK-7654] [SQL] Move insertInto into reader/writer interface. This one continues the work of https://github.com/apache/spark/pull/6216. Author: Yin Huai yh...@databricks.com Author: Reynold Xin r...@databricks.com Closes #6366 from yhuai/insert and squashes the following commits: 3d717fb [Yin Huai] Use insertInto to handle the casue when table exists and Append is used for saveAsTable. 56d2540 [Yin Huai] Add PreWriteCheck to HiveContext's analyzer. c636e35 [Yin Huai] Remove unnecessary empty lines. cf83837 [Yin Huai] Move insertInto to write. Also, remove the partition columns from InsertIntoHadoopFsRelation. 0841a54 [Reynold Xin] Removed experimental tag for deprecated methods. 33ed8ef [Reynold Xin] [SPARK-7654][SQL] Move insertInto into reader/writer interface. (cherry picked from commit 2b7e63585d61be2dab78b70af3867cda3983d5b1) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8d6d8a53 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8d6d8a53 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8d6d8a53 Branch: refs/heads/branch-1.4 Commit: 8d6d8a538c46d9b41db1b62ebe7b7c038fdb057c Parents: c6e5742 Author: Yin Huai yh...@databricks.com Authored: Sat May 23 09:48:20 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sat May 23 09:48:30 2015 -0700 -- .../scala/org/apache/spark/sql/DataFrame.scala | 52 +++ .../org/apache/spark/sql/DataFrameReader.scala | 18 +- .../org/apache/spark/sql/DataFrameWriter.scala | 66 +--- .../spark/sql/parquet/ParquetTableSupport.scala | 2 +- .../spark/sql/sources/DataSourceStrategy.scala | 5 +- .../org/apache/spark/sql/sources/commands.scala | 2 +- .../org/apache/spark/sql/sources/ddl.scala | 1 - .../org/apache/spark/sql/sources/rules.scala| 19 +- .../org/apache/spark/sql/hive/HiveContext.scala | 4 ++ .../sql/hive/InsertIntoHiveTableSuite.scala | 6 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 8 +-- .../sql/hive/execution/SQLQuerySuite.scala | 8 +-- .../apache/spark/sql/hive/parquetSuites.scala | 4 +- .../sql/sources/hadoopFsRelationSuites.scala| 10 --- 14 files changed, 116 insertions(+), 89 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8d6d8a53/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala index 3ec1c4a..f968577 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/DataFrame.scala @@ -1395,28 +1395,6 @@ class DataFrame private[sql]( def write: DataFrameWriter = new DataFrameWriter(this) /** - * :: Experimental :: - * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. - * @group output - * @since 1.3.0 - */ - @Experimental - def insertInto(tableName: String, overwrite: Boolean): Unit = { -sqlContext.executePlan(InsertIntoTable(UnresolvedRelation(Seq(tableName)), - Map.empty, logicalPlan, overwrite, ifNotExists = false)).toRdd - } - - /** - * :: Experimental :: - * Adds the rows from this RDD to the specified table. - * Throws an exception if the table already exists. - * @group output - * @since 1.3.0 - */ - @Experimental - def insertInto(tableName: String): Unit = insertInto(tableName, overwrite = false) - - /** * Returns the content of the [[DataFrame]] as a RDD of JSON strings. * @group rdd * @since 1.3.0 @@ -1551,13 +1529,7 @@ class DataFrame private[sql]( */ @deprecated(Use write.mode(mode).saveAsTable(tableName), 1.4.0) def saveAsTable(tableName: String, mode: SaveMode): Unit = { -if (sqlContext.catalog.tableExists(Seq(tableName)) mode == SaveMode.Append) { - // If table already exists and the save mode is Append, - // we will just call insertInto to append the contents of this DataFrame. - insertInto(tableName, overwrite = false) -} else { - write.mode(mode).saveAsTable(tableName) -} +write.mode(mode).saveAsTable(tableName) } /** @@ -1713,9 +1685,29 @@ class DataFrame private[sql]( write.format(source).mode(mode).options(options).save() } + + /** + * Adds the rows from this RDD to the specified table, optionally overwriting the existing data. + * @group output + */ + @deprecated(Use write.mode(SaveMode.Append|SaveMode.Overwrite).saveAsTable(tableName), 1.4.0) + def insertInto(tableName: String, overwrite: Boolean): Unit
spark git commit: [SPARK-7749] [SQL] Fixes partition discovery for non-partitioned tables
Repository: spark Updated Branches: refs/heads/master 13348e21b - 8730fbb47 [SPARK-7749] [SQL] Fixes partition discovery for non-partitioned tables When no partition columns can be found, we should have an empty `PartitionSpec`, rather than a `PartitionSpec` with empty partition columns. This PR together with #6285 should fix SPARK-7749. Author: Cheng Lian l...@databricks.com Author: Yin Huai yh...@databricks.com Closes #6287 from liancheng/spark-7749 and squashes the following commits: a799ff3 [Cheng Lian] Adds test cases for SPARK-7749 c4949be [Cheng Lian] Minor refactoring, and tolerant _TEMPORARY directory name 5aa87ea [Yin Huai] Make parsePartitions more robust. fc56656 [Cheng Lian] Returns empty PartitionSpec if no partition columns can be inferred 19ae41e [Cheng Lian] Don't list base directory as leaf directory Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/8730fbb4 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/8730fbb4 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/8730fbb4 Branch: refs/heads/master Commit: 8730fbb47b09fcf955fe16dd03b75596db6d53b6 Parents: 13348e2 Author: Cheng Lian l...@databricks.com Authored: Thu May 21 10:56:17 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 10:56:17 2015 -0700 -- .../spark/sql/sources/PartitioningUtils.scala | 84 +--- .../apache/spark/sql/sources/interfaces.scala | 7 +- .../ParquetPartitionDiscoverySuite.scala| 49 ++-- .../apache/spark/sql/hive/parquetSuites.scala | 51 +++- 4 files changed, 150 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/8730fbb4/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala index 8f8138d..e0ead23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala @@ -33,6 +33,10 @@ private[sql] case class Partition(values: Row, path: String) private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) +private[sql] object PartitionSpec { + val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[Partition]) +} + private[sql] object PartitioningUtils { // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since sql/core doesn't // depend on Hive. @@ -68,20 +72,37 @@ private[sql] object PartitioningUtils { private[sql] def parsePartitions( paths: Seq[Path], defaultPartitionName: String): PartitionSpec = { -val partitionValues = resolvePartitions(paths.flatMap(parsePartition(_, defaultPartitionName))) -val fields = { - val (PartitionValues(columnNames, literals)) = partitionValues.head - columnNames.zip(literals).map { case (name, Literal(_, dataType)) = -StructField(name, dataType, nullable = true) - } +// First, we need to parse every partition's path and see if we can find partition values. +val pathsWithPartitionValues = paths.flatMap { path = + parsePartition(path, defaultPartitionName).map(path - _) } -val partitions = partitionValues.zip(paths).map { - case (PartitionValues(_, literals), path) = -Partition(Row(literals.map(_.value): _*), path.toString) -} +if (pathsWithPartitionValues.isEmpty) { + // This dataset is not partitioned. + PartitionSpec.emptySpec +} else { + // This dataset is partitioned. We need to check whether all partitions have the same + // partition columns and resolve potential type conflicts. + val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues.map(_._2)) + + // Creates the StructType which represents the partition columns. + val fields = { +val PartitionValues(columnNames, literals) = resolvedPartitionValues.head +columnNames.zip(literals).map { case (name, Literal(_, dataType)) = + // We always assume partition columns are nullable since we've no idea whether null values + // will be appended in the future. + StructField(name, dataType, nullable = true) +} + } + + // Finally, we create `Partition`s based on paths and resolved partition values. + val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map { +case (PartitionValues(_, literals), (path, _)) = + Partition(Row.fromSeq(literals.map(_.value)), path.toString) + } -
spark git commit: [SPARK-7749] [SQL] Fixes partition discovery for non-partitioned tables
Repository: spark Updated Branches: refs/heads/branch-1.4 b97a8053a - 70d9839cf [SPARK-7749] [SQL] Fixes partition discovery for non-partitioned tables When no partition columns can be found, we should have an empty `PartitionSpec`, rather than a `PartitionSpec` with empty partition columns. This PR together with #6285 should fix SPARK-7749. Author: Cheng Lian l...@databricks.com Author: Yin Huai yh...@databricks.com Closes #6287 from liancheng/spark-7749 and squashes the following commits: a799ff3 [Cheng Lian] Adds test cases for SPARK-7749 c4949be [Cheng Lian] Minor refactoring, and tolerant _TEMPORARY directory name 5aa87ea [Yin Huai] Make parsePartitions more robust. fc56656 [Cheng Lian] Returns empty PartitionSpec if no partition columns can be inferred 19ae41e [Cheng Lian] Don't list base directory as leaf directory (cherry picked from commit 8730fbb47b09fcf955fe16dd03b75596db6d53b6) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/70d9839c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/70d9839c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/70d9839c Branch: refs/heads/branch-1.4 Commit: 70d9839cf3b553573c43d884b45a8b942e2f4770 Parents: b97a805 Author: Cheng Lian l...@databricks.com Authored: Thu May 21 10:56:17 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 10:56:26 2015 -0700 -- .../spark/sql/sources/PartitioningUtils.scala | 84 +--- .../apache/spark/sql/sources/interfaces.scala | 7 +- .../ParquetPartitionDiscoverySuite.scala| 49 ++-- .../apache/spark/sql/hive/parquetSuites.scala | 51 +++- 4 files changed, 150 insertions(+), 41 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/70d9839c/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala index 8f8138d..e0ead23 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/sources/PartitioningUtils.scala @@ -33,6 +33,10 @@ private[sql] case class Partition(values: Row, path: String) private[sql] case class PartitionSpec(partitionColumns: StructType, partitions: Seq[Partition]) +private[sql] object PartitionSpec { + val emptySpec = PartitionSpec(StructType(Seq.empty[StructField]), Seq.empty[Partition]) +} + private[sql] object PartitioningUtils { // This duplicates default value of Hive `ConfVars.DEFAULTPARTITIONNAME`, since sql/core doesn't // depend on Hive. @@ -68,20 +72,37 @@ private[sql] object PartitioningUtils { private[sql] def parsePartitions( paths: Seq[Path], defaultPartitionName: String): PartitionSpec = { -val partitionValues = resolvePartitions(paths.flatMap(parsePartition(_, defaultPartitionName))) -val fields = { - val (PartitionValues(columnNames, literals)) = partitionValues.head - columnNames.zip(literals).map { case (name, Literal(_, dataType)) = -StructField(name, dataType, nullable = true) - } +// First, we need to parse every partition's path and see if we can find partition values. +val pathsWithPartitionValues = paths.flatMap { path = + parsePartition(path, defaultPartitionName).map(path - _) } -val partitions = partitionValues.zip(paths).map { - case (PartitionValues(_, literals), path) = -Partition(Row(literals.map(_.value): _*), path.toString) -} +if (pathsWithPartitionValues.isEmpty) { + // This dataset is not partitioned. + PartitionSpec.emptySpec +} else { + // This dataset is partitioned. We need to check whether all partitions have the same + // partition columns and resolve potential type conflicts. + val resolvedPartitionValues = resolvePartitions(pathsWithPartitionValues.map(_._2)) + + // Creates the StructType which represents the partition columns. + val fields = { +val PartitionValues(columnNames, literals) = resolvedPartitionValues.head +columnNames.zip(literals).map { case (name, Literal(_, dataType)) = + // We always assume partition columns are nullable since we've no idea whether null values + // will be appended in the future. + StructField(name, dataType, nullable = true) +} + } + + // Finally, we create `Partition`s based on paths and resolved partition values. + val partitions = resolvedPartitionValues.zip(pathsWithPartitionValues).map { +case (PartitionValues(_,
spark git commit: [SPARK-7763] [SPARK-7616] [SQL] Persists partition columns into metastore
Repository: spark Updated Branches: refs/heads/branch-1.4 33e0e - 96c82515b [SPARK-7763] [SPARK-7616] [SQL] Persists partition columns into metastore Author: Yin Huai yh...@databricks.com Author: Cheng Lian l...@databricks.com Closes #6285 from liancheng/spark-7763 and squashes the following commits: bb2829d [Yin Huai] Fix hashCode. d677f7d [Cheng Lian] Fixes Scala style issue 44b283f [Cheng Lian] Adds test case for SPARK-7616 6733276 [Yin Huai] Fix a bug that potentially causes https://issues.apache.org/jira/browse/SPARK-7616. 6cabf3c [Yin Huai] Update unit test. 7e02910 [Yin Huai] Use metastore partition columns and do not hijack maybePartitionSpec. e9a03ec [Cheng Lian] Persists partition columns into metastore (cherry picked from commit 30f3f556f7161a49baf145c0cbba8c088b512a6a) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/96c82515 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/96c82515 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/96c82515 Branch: refs/heads/branch-1.4 Commit: 96c82515b8b1ec821513ffa1015f928ef534680b Parents: 33e Author: Yin Huai yh...@databricks.com Authored: Thu May 21 13:51:40 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 13:51:49 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 26 --- .../org/apache/spark/sql/sources/commands.scala | 2 + .../org/apache/spark/sql/sources/ddl.scala | 19 ++-- .../apache/spark/sql/sources/interfaces.scala | 31 +++-- .../apache/spark/sql/test/SQLTestUtils.scala| 7 +++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 49 .../spark/sql/hive/execution/commands.scala | 2 + .../apache/spark/sql/hive/orc/OrcRelation.scala | 35 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 30 .../apache/spark/sql/hive/parquetSuites.scala | 28 +-- .../spark/sql/sources/SimpleTextRelation.scala | 2 +- .../sql/sources/hadoopFsRelationSuites.scala| 36 -- 12 files changed, 211 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/96c82515/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index c35b7ef..32986aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -49,8 +49,7 @@ private[sql] class DefaultSource extends HadoopFsRelationProvider { schema: Option[StructType], partitionColumns: Option[StructType], parameters: Map[String, String]): HadoopFsRelation = { -val partitionSpec = partitionColumns.map(PartitionSpec(_, Seq.empty)) -new ParquetRelation2(paths, schema, partitionSpec, parameters)(sqlContext) +new ParquetRelation2(paths, schema, None, partitionColumns, parameters)(sqlContext) } } @@ -118,12 +117,28 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext private[sql] class ParquetRelation2( override val paths: Array[String], private val maybeDataSchema: Option[StructType], +// This is for metastore conversion. private val maybePartitionSpec: Option[PartitionSpec], +override val userDefinedPartitionColumns: Option[StructType], parameters: Map[String, String])( val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec) with Logging { + private[sql] def this( + paths: Array[String], + maybeDataSchema: Option[StructType], + maybePartitionSpec: Option[PartitionSpec], + parameters: Map[String, String])( + sqlContext: SQLContext) = { +this( + paths, + maybeDataSchema, + maybePartitionSpec, + maybePartitionSpec.map(_.partitionColumns), + parameters)(sqlContext) + } + // Should we merge schemas from all Parquet part-files? private val shouldMergeSchemas = parameters.getOrElse(ParquetRelation2.MERGE_SCHEMA, true).toBoolean @@ -161,7 +176,7 @@ private[sql] class ParquetRelation2( Boolean.box(shouldMergeSchemas), paths.toSet, maybeDataSchema, -maybePartitionSpec) +partitionColumns) } else { Objects.hashCode( Boolean.box(shouldMergeSchemas), @@ -169,7 +184,7 @@ private[sql] class ParquetRelation2( dataSchema, schema, maybeDataSchema, -maybePartitionSpec) +partitionColumns) } } @@ -185,9 +200,6 @@ private[sql] class ParquetRelation2(
spark git commit: [SPARK-7763] [SPARK-7616] [SQL] Persists partition columns into metastore
Repository: spark Updated Branches: refs/heads/master 311fab6f1 - 30f3f556f [SPARK-7763] [SPARK-7616] [SQL] Persists partition columns into metastore Author: Yin Huai yh...@databricks.com Author: Cheng Lian l...@databricks.com Closes #6285 from liancheng/spark-7763 and squashes the following commits: bb2829d [Yin Huai] Fix hashCode. d677f7d [Cheng Lian] Fixes Scala style issue 44b283f [Cheng Lian] Adds test case for SPARK-7616 6733276 [Yin Huai] Fix a bug that potentially causes https://issues.apache.org/jira/browse/SPARK-7616. 6cabf3c [Yin Huai] Update unit test. 7e02910 [Yin Huai] Use metastore partition columns and do not hijack maybePartitionSpec. e9a03ec [Cheng Lian] Persists partition columns into metastore Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/30f3f556 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/30f3f556 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/30f3f556 Branch: refs/heads/master Commit: 30f3f556f7161a49baf145c0cbba8c088b512a6a Parents: 311fab6 Author: Yin Huai yh...@databricks.com Authored: Thu May 21 13:51:40 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 13:51:40 2015 -0700 -- .../apache/spark/sql/parquet/newParquet.scala | 26 --- .../org/apache/spark/sql/sources/commands.scala | 2 + .../org/apache/spark/sql/sources/ddl.scala | 19 ++-- .../apache/spark/sql/sources/interfaces.scala | 31 +++-- .../apache/spark/sql/test/SQLTestUtils.scala| 7 +++ .../spark/sql/hive/HiveMetastoreCatalog.scala | 49 .../spark/sql/hive/execution/commands.scala | 2 + .../apache/spark/sql/hive/orc/OrcRelation.scala | 35 +- .../sql/hive/MetastoreDataSourcesSuite.scala| 30 .../apache/spark/sql/hive/parquetSuites.scala | 28 +-- .../spark/sql/sources/SimpleTextRelation.scala | 2 +- .../sql/sources/hadoopFsRelationSuites.scala| 36 -- 12 files changed, 211 insertions(+), 56 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/30f3f556/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index c35b7ef..32986aa 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -49,8 +49,7 @@ private[sql] class DefaultSource extends HadoopFsRelationProvider { schema: Option[StructType], partitionColumns: Option[StructType], parameters: Map[String, String]): HadoopFsRelation = { -val partitionSpec = partitionColumns.map(PartitionSpec(_, Seq.empty)) -new ParquetRelation2(paths, schema, partitionSpec, parameters)(sqlContext) +new ParquetRelation2(paths, schema, None, partitionColumns, parameters)(sqlContext) } } @@ -118,12 +117,28 @@ private[sql] class ParquetOutputWriter(path: String, context: TaskAttemptContext private[sql] class ParquetRelation2( override val paths: Array[String], private val maybeDataSchema: Option[StructType], +// This is for metastore conversion. private val maybePartitionSpec: Option[PartitionSpec], +override val userDefinedPartitionColumns: Option[StructType], parameters: Map[String, String])( val sqlContext: SQLContext) extends HadoopFsRelation(maybePartitionSpec) with Logging { + private[sql] def this( + paths: Array[String], + maybeDataSchema: Option[StructType], + maybePartitionSpec: Option[PartitionSpec], + parameters: Map[String, String])( + sqlContext: SQLContext) = { +this( + paths, + maybeDataSchema, + maybePartitionSpec, + maybePartitionSpec.map(_.partitionColumns), + parameters)(sqlContext) + } + // Should we merge schemas from all Parquet part-files? private val shouldMergeSchemas = parameters.getOrElse(ParquetRelation2.MERGE_SCHEMA, true).toBoolean @@ -161,7 +176,7 @@ private[sql] class ParquetRelation2( Boolean.box(shouldMergeSchemas), paths.toSet, maybeDataSchema, -maybePartitionSpec) +partitionColumns) } else { Objects.hashCode( Boolean.box(shouldMergeSchemas), @@ -169,7 +184,7 @@ private[sql] class ParquetRelation2( dataSchema, schema, maybeDataSchema, -maybePartitionSpec) +partitionColumns) } } @@ -185,9 +200,6 @@ private[sql] class ParquetRelation2( override def sizeInBytes: Long = metadataCache.dataStatuses.map(_.getLen).sum - override def
spark git commit: [SPARK-7718] [SQL] Speed up partitioning by avoiding closure cleaning
Repository: spark Updated Branches: refs/heads/branch-1.4 c9a80fc40 - ba04b5236 [SPARK-7718] [SQL] Speed up partitioning by avoiding closure cleaning According to yhuai we spent 6-7 seconds cleaning closures in a partitioning job that takes 12 seconds. Since we provide these closures in Spark we know for sure they are serializable, so we can bypass the cleaning. Author: Andrew Or and...@databricks.com Closes #6256 from andrewor14/sql-partition-speed-up and squashes the following commits: a82b451 [Andrew Or] Fix style 10f7e3e [Andrew Or] Avoid getting call sites and cleaning closures 17e2943 [Andrew Or] Merge branch 'master' of github.com:apache/spark into sql-partition-speed-up 523f042 [Andrew Or] Skip unnecessary Utils.getCallSites too f7fe143 [Andrew Or] Avoid unnecessary closure cleaning (cherry picked from commit 5287eec5a6948c0c6e0baaebf35f512324c0679a) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ba04b523 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ba04b523 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ba04b523 Branch: refs/heads/branch-1.4 Commit: ba04b52360a8adad790f30c77c1f6ef76f17ad2e Parents: c9a80fc Author: Andrew Or and...@databricks.com Authored: Thu May 21 14:33:11 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 14:33:24 2015 -0700 -- .../scala/org/apache/spark/util/Utils.scala | 18 .../apache/spark/sql/parquet/newParquet.scala | 98 ++-- .../spark/sql/sources/DataSourceStrategy.scala | 18 +++- .../spark/sql/sources/SqlNewHadoopRDD.scala | 4 - 4 files changed, 83 insertions(+), 55 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ba04b523/core/src/main/scala/org/apache/spark/util/Utils.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6a7d1fa..b7a2473 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2201,6 +2201,24 @@ private[spark] object Utils extends Logging { shutdownHooks.remove(ref) } + /** + * To avoid calling `Utils.getCallSite` for every single RDD we create in the body, + * set a dummy call site that RDDs use instead. This is for performance optimization. + */ + def withDummyCallSite[T](sc: SparkContext)(body: = T): T = { +val oldShortCallSite = sc.getLocalProperty(CallSite.SHORT_FORM) +val oldLongCallSite = sc.getLocalProperty(CallSite.LONG_FORM) +try { + sc.setLocalProperty(CallSite.SHORT_FORM, ) + sc.setLocalProperty(CallSite.LONG_FORM, ) + body +} finally { + // Restore the old ones here + sc.setLocalProperty(CallSite.SHORT_FORM, oldShortCallSite) + sc.setLocalProperty(CallSite.LONG_FORM, oldLongCallSite) +} + } + } private [util] class SparkShutdownHookManager { http://git-wip-us.apache.org/repos/asf/spark/blob/ba04b523/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 32986aa..cb1e608 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -33,6 +33,7 @@ import parquet.hadoop._ import parquet.hadoop.metadata.CompressionCodecName import parquet.hadoop.util.ContextUtil +import org.apache.spark.{Partition = SparkPartition, SerializableWritable, Logging, SparkException} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD._ @@ -40,7 +41,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.{Row, SQLConf, SQLContext} -import org.apache.spark.{Partition = SparkPartition, SparkEnv, SerializableWritable, Logging, SparkException} +import org.apache.spark.util.Utils private[sql] class DefaultSource extends HadoopFsRelationProvider { override def createRelation( @@ -264,57 +265,58 @@ private[sql] class ParquetRelation2( val footers = inputFiles.map(f = metadataCache.footers(f.getPath)) -// TODO Stop using `FilteringParquetRowInputFormat` and overriding `getPartition`. -// After upgrading to Parquet 1.6.0, we should be able to stop caching `FileStatus` objects and -// footers. Especially when a global arbitrative schema (either from metastore
spark git commit: [SPARK-7718] [SQL] Speed up partitioning by avoiding closure cleaning
Repository: spark Updated Branches: refs/heads/master 6b18cdc1b - 5287eec5a [SPARK-7718] [SQL] Speed up partitioning by avoiding closure cleaning According to yhuai we spent 6-7 seconds cleaning closures in a partitioning job that takes 12 seconds. Since we provide these closures in Spark we know for sure they are serializable, so we can bypass the cleaning. Author: Andrew Or and...@databricks.com Closes #6256 from andrewor14/sql-partition-speed-up and squashes the following commits: a82b451 [Andrew Or] Fix style 10f7e3e [Andrew Or] Avoid getting call sites and cleaning closures 17e2943 [Andrew Or] Merge branch 'master' of github.com:apache/spark into sql-partition-speed-up 523f042 [Andrew Or] Skip unnecessary Utils.getCallSites too f7fe143 [Andrew Or] Avoid unnecessary closure cleaning Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5287eec5 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5287eec5 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5287eec5 Branch: refs/heads/master Commit: 5287eec5a6948c0c6e0baaebf35f512324c0679a Parents: 6b18cdc Author: Andrew Or and...@databricks.com Authored: Thu May 21 14:33:11 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 14:33:11 2015 -0700 -- .../scala/org/apache/spark/util/Utils.scala | 18 .../apache/spark/sql/parquet/newParquet.scala | 98 ++-- .../spark/sql/sources/DataSourceStrategy.scala | 18 +++- .../spark/sql/sources/SqlNewHadoopRDD.scala | 4 - 4 files changed, 83 insertions(+), 55 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5287eec5/core/src/main/scala/org/apache/spark/util/Utils.scala -- diff --git a/core/src/main/scala/org/apache/spark/util/Utils.scala b/core/src/main/scala/org/apache/spark/util/Utils.scala index 6a7d1fa..b7a2473 100644 --- a/core/src/main/scala/org/apache/spark/util/Utils.scala +++ b/core/src/main/scala/org/apache/spark/util/Utils.scala @@ -2201,6 +2201,24 @@ private[spark] object Utils extends Logging { shutdownHooks.remove(ref) } + /** + * To avoid calling `Utils.getCallSite` for every single RDD we create in the body, + * set a dummy call site that RDDs use instead. This is for performance optimization. + */ + def withDummyCallSite[T](sc: SparkContext)(body: = T): T = { +val oldShortCallSite = sc.getLocalProperty(CallSite.SHORT_FORM) +val oldLongCallSite = sc.getLocalProperty(CallSite.LONG_FORM) +try { + sc.setLocalProperty(CallSite.SHORT_FORM, ) + sc.setLocalProperty(CallSite.LONG_FORM, ) + body +} finally { + // Restore the old ones here + sc.setLocalProperty(CallSite.SHORT_FORM, oldShortCallSite) + sc.setLocalProperty(CallSite.LONG_FORM, oldLongCallSite) +} + } + } private [util] class SparkShutdownHookManager { http://git-wip-us.apache.org/repos/asf/spark/blob/5287eec5/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala index 32986aa..cb1e608 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/parquet/newParquet.scala @@ -33,6 +33,7 @@ import parquet.hadoop._ import parquet.hadoop.metadata.CompressionCodecName import parquet.hadoop.util.ContextUtil +import org.apache.spark.{Partition = SparkPartition, SerializableWritable, Logging, SparkException} import org.apache.spark.broadcast.Broadcast import org.apache.spark.deploy.SparkHadoopUtil import org.apache.spark.rdd.RDD._ @@ -40,7 +41,7 @@ import org.apache.spark.rdd.RDD import org.apache.spark.sql.sources._ import org.apache.spark.sql.types.{DataType, StructType} import org.apache.spark.sql.{Row, SQLConf, SQLContext} -import org.apache.spark.{Partition = SparkPartition, SparkEnv, SerializableWritable, Logging, SparkException} +import org.apache.spark.util.Utils private[sql] class DefaultSource extends HadoopFsRelationProvider { override def createRelation( @@ -264,57 +265,58 @@ private[sql] class ParquetRelation2( val footers = inputFiles.map(f = metadataCache.footers(f.getPath)) -// TODO Stop using `FilteringParquetRowInputFormat` and overriding `getPartition`. -// After upgrading to Parquet 1.6.0, we should be able to stop caching `FileStatus` objects and -// footers. Especially when a global arbitrative schema (either from metastore or data source -// DDL) is available. -new SqlNewHadoopRDD( - sc = sqlContext.sparkContext
spark git commit: [SPARK-7565] [SQL] fix MapType in JsonRDD
Repository: spark Updated Branches: refs/heads/master feb3a9d3f - a25c1ab8f [SPARK-7565] [SQL] fix MapType in JsonRDD The key of Map in JsonRDD should be converted into UTF8String (also failed records), Thanks to yhuai viirya Closes #6084 Author: Davies Liu dav...@databricks.com Closes #6299 from davies/string_in_json and squashes the following commits: 0dbf559 [Davies Liu] improve test, fix corrupt record 6836a80 [Davies Liu] move unit tests into Scala b97af11 [Davies Liu] fix MapType in JsonRDD Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/a25c1ab8 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/a25c1ab8 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/a25c1ab8 Branch: refs/heads/master Commit: a25c1ab8f04a4e19d82ff4c18a0b1689d8b3ddac Parents: feb3a9d Author: Davies Liu dav...@databricks.com Authored: Thu May 21 09:58:47 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 09:58:47 2015 -0700 -- .../apache/spark/sql/json/JacksonParser.scala | 8 +++--- .../org/apache/spark/sql/json/JsonRDD.scala | 16 ++- .../org/apache/spark/sql/json/JsonSuite.scala | 28 +++- 3 files changed, 41 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/a25c1ab8/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala index 8161151..0e22375 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala @@ -150,10 +150,10 @@ private[sql] object JacksonParser { private def convertMap( factory: JsonFactory, parser: JsonParser, - valueType: DataType): Map[String, Any] = { -val builder = Map.newBuilder[String, Any] + valueType: DataType): Map[UTF8String, Any] = { +val builder = Map.newBuilder[UTF8String, Any] while (nextUntil(parser, JsonToken.END_OBJECT)) { - builder += parser.getCurrentName - convertField(factory, parser, valueType) + builder += UTF8String(parser.getCurrentName) - convertField(factory, parser, valueType) } builder.result() @@ -181,7 +181,7 @@ private[sql] object JacksonParser { val row = new GenericMutableRow(schema.length) for (corruptIndex - schema.getFieldIndex(columnNameOfCorruptRecords)) { require(schema(corruptIndex).dataType == StringType) -row.update(corruptIndex, record) +row.update(corruptIndex, UTF8String(record)) } Seq(row) http://git-wip-us.apache.org/repos/asf/spark/blob/a25c1ab8/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 4c32710..037a6d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -20,18 +20,18 @@ package org.apache.spark.sql.json import java.sql.Timestamp import scala.collection.Map -import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} +import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} -import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException} +import com.fasterxml.jackson.core.JsonProcessingException import com.fasterxml.jackson.databind.ObjectMapper +import org.apache.spark.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.util.DateUtils import org.apache.spark.sql.types._ -import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { @@ -318,7 +318,8 @@ private[sql] object JsonRDD extends Logging { parsed } catch { - case e: JsonProcessingException = Map(columnNameOfCorruptRecords - record) :: Nil + case e: JsonProcessingException = +Map(columnNameOfCorruptRecords - UTF8String(record)) :: Nil } } }) @@ -422,7 +423,10 @@ private[sql] object JsonRDD extends Logging { value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case MapType(StringType, valueType, _) = val map = value.asInstanceOf[Map[String, Any]] - map.mapValues
spark git commit: [SPARK-7565] [SQL] fix MapType in JsonRDD
Repository: spark Updated Branches: refs/heads/branch-1.4 f0e421351 - 3aa618510 [SPARK-7565] [SQL] fix MapType in JsonRDD The key of Map in JsonRDD should be converted into UTF8String (also failed records), Thanks to yhuai viirya Closes #6084 Author: Davies Liu dav...@databricks.com Closes #6299 from davies/string_in_json and squashes the following commits: 0dbf559 [Davies Liu] improve test, fix corrupt record 6836a80 [Davies Liu] move unit tests into Scala b97af11 [Davies Liu] fix MapType in JsonRDD (cherry picked from commit a25c1ab8f04a4e19d82ff4c18a0b1689d8b3ddac) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3aa61851 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3aa61851 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3aa61851 Branch: refs/heads/branch-1.4 Commit: 3aa618510167ef72b4107d964a490be9d90da70d Parents: f0e4213 Author: Davies Liu dav...@databricks.com Authored: Thu May 21 09:58:47 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 09:58:57 2015 -0700 -- .../apache/spark/sql/json/JacksonParser.scala | 8 +++--- .../org/apache/spark/sql/json/JsonRDD.scala | 16 ++- .../org/apache/spark/sql/json/JsonSuite.scala | 28 +++- 3 files changed, 41 insertions(+), 11 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3aa61851/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala index 8161151..0e22375 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JacksonParser.scala @@ -150,10 +150,10 @@ private[sql] object JacksonParser { private def convertMap( factory: JsonFactory, parser: JsonParser, - valueType: DataType): Map[String, Any] = { -val builder = Map.newBuilder[String, Any] + valueType: DataType): Map[UTF8String, Any] = { +val builder = Map.newBuilder[UTF8String, Any] while (nextUntil(parser, JsonToken.END_OBJECT)) { - builder += parser.getCurrentName - convertField(factory, parser, valueType) + builder += UTF8String(parser.getCurrentName) - convertField(factory, parser, valueType) } builder.result() @@ -181,7 +181,7 @@ private[sql] object JacksonParser { val row = new GenericMutableRow(schema.length) for (corruptIndex - schema.getFieldIndex(columnNameOfCorruptRecords)) { require(schema(corruptIndex).dataType == StringType) -row.update(corruptIndex, record) +row.update(corruptIndex, UTF8String(record)) } Seq(row) http://git-wip-us.apache.org/repos/asf/spark/blob/3aa61851/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala index 4c32710..037a6d6 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/json/JsonRDD.scala @@ -20,18 +20,18 @@ package org.apache.spark.sql.json import java.sql.Timestamp import scala.collection.Map -import scala.collection.convert.Wrappers.{JMapWrapper, JListWrapper} +import scala.collection.convert.Wrappers.{JListWrapper, JMapWrapper} -import com.fasterxml.jackson.core.{JsonGenerator, JsonProcessingException} +import com.fasterxml.jackson.core.JsonProcessingException import com.fasterxml.jackson.databind.ObjectMapper +import org.apache.spark.Logging import org.apache.spark.rdd.RDD +import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.analysis.HiveTypeCoercion import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.ScalaReflection import org.apache.spark.sql.catalyst.util.DateUtils import org.apache.spark.sql.types._ -import org.apache.spark.Logging private[sql] object JsonRDD extends Logging { @@ -318,7 +318,8 @@ private[sql] object JsonRDD extends Logging { parsed } catch { - case e: JsonProcessingException = Map(columnNameOfCorruptRecords - record) :: Nil + case e: JsonProcessingException = +Map(columnNameOfCorruptRecords - UTF8String(record)) :: Nil } } }) @@ -422,7 +423,10 @@ private[sql] object JsonRDD extends Logging { value.asInstanceOf[Seq[Any]].map(enforceCorrectType(_, elementType)) case
spark git commit: [SPARK-7320] [SQL] [Minor] Move the testData into beforeAll()
Repository: spark Updated Branches: refs/heads/master 1ee8eb431 - feb3a9d3f [SPARK-7320] [SQL] [Minor] Move the testData into beforeAll() Follow up of #6340, to avoid the test report missing once it fails. Author: Cheng Hao hao.ch...@intel.com Closes #6312 from chenghao-intel/rollup_minor and squashes the following commits: b03a25f [Cheng Hao] simplify the testData instantiation 09b7e8b [Cheng Hao] move the testData into beforeAll() Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/feb3a9d3 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/feb3a9d3 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/feb3a9d3 Branch: refs/heads/master Commit: feb3a9d3f81f19850fddbd9639823f59a60efa52 Parents: 1ee8eb4 Author: Cheng Hao hao.ch...@intel.com Authored: Thu May 21 09:28:00 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 09:28:00 2015 -0700 -- .../spark/sql/hive/HiveDataFrameAnalyticsSuite.scala | 10 +++--- 1 file changed, 3 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/feb3a9d3/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala index 99de146..fb10f85 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala @@ -17,25 +17,21 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.scalatest.BeforeAndAfterAll -case class TestData2Int(a: Int, b: Int) - // TODO ideally we should put the test suite into the package `sql`, as // `hive` package is optional in compiling, however, `SQLContext.sql` doesn't // support the `cube` or `rollup` yet. class HiveDataFrameAnalyticsSuite extends QueryTest with BeforeAndAfterAll { - val testData = -TestHive.sparkContext.parallelize( - TestData2Int(1, 2) :: -TestData2Int(2, 4) :: Nil).toDF() + private var testData: DataFrame = _ override def beforeAll() { +testData = Seq((1, 2), (2, 4)).toDF(a, b) TestHive.registerDataFrameAsTable(testData, mytable) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-7320] [SQL] [Minor] Move the testData into beforeAll()
Repository: spark Updated Branches: refs/heads/branch-1.4 f08c6f319 - f0e421351 [SPARK-7320] [SQL] [Minor] Move the testData into beforeAll() Follow up of #6340, to avoid the test report missing once it fails. Author: Cheng Hao hao.ch...@intel.com Closes #6312 from chenghao-intel/rollup_minor and squashes the following commits: b03a25f [Cheng Hao] simplify the testData instantiation 09b7e8b [Cheng Hao] move the testData into beforeAll() (cherry picked from commit feb3a9d3f81f19850fddbd9639823f59a60efa52) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/f0e42135 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/f0e42135 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/f0e42135 Branch: refs/heads/branch-1.4 Commit: f0e42135175f5fe61b99b88b327f334e9df9a2bc Parents: f08c6f3 Author: Cheng Hao hao.ch...@intel.com Authored: Thu May 21 09:28:00 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu May 21 09:28:10 2015 -0700 -- .../spark/sql/hive/HiveDataFrameAnalyticsSuite.scala | 10 +++--- 1 file changed, 3 insertions(+), 7 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/f0e42135/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala index 99de146..fb10f85 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveDataFrameAnalyticsSuite.scala @@ -17,25 +17,21 @@ package org.apache.spark.sql.hive -import org.apache.spark.sql.QueryTest +import org.apache.spark.sql.{DataFrame, QueryTest} import org.apache.spark.sql.functions._ import org.apache.spark.sql.hive.test.TestHive import org.apache.spark.sql.hive.test.TestHive._ import org.apache.spark.sql.hive.test.TestHive.implicits._ import org.scalatest.BeforeAndAfterAll -case class TestData2Int(a: Int, b: Int) - // TODO ideally we should put the test suite into the package `sql`, as // `hive` package is optional in compiling, however, `SQLContext.sql` doesn't // support the `cube` or `rollup` yet. class HiveDataFrameAnalyticsSuite extends QueryTest with BeforeAndAfterAll { - val testData = -TestHive.sparkContext.parallelize( - TestData2Int(1, 2) :: -TestData2Int(2, 4) :: Nil).toDF() + private var testData: DataFrame = _ override def beforeAll() { +testData = Seq((1, 2), (2, 4)).toDF(a, b) TestHive.registerDataFrameAsTable(testData, mytable) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-2205] [SQL] Avoid unnecessary exchange operators in multi-way joins
Repository: spark Updated Branches: refs/heads/master 30e89111d - 114ff926f [SPARK-2205] [SQL] Avoid unnecessary exchange operators in multi-way joins This PR adds `PartitioningCollection`, which is used to represent the `outputPartitioning` for SparkPlans with multiple children (e.g. `ShuffledHashJoin`). So, a `SparkPlan` can have multiple descriptions of its partitioning schemes. Taking `ShuffledHashJoin` as an example, it has two descriptions of its partitioning schemes, i.e. `left.outputPartitioning` and `right.outputPartitioning`. So when we have a query like `select * from t1 join t2 on (t1.x = t2.x) join t3 on (t2.x = t3.x)` will only have three Exchange operators (when shuffled joins are needed) instead of four. The code in this PR was authored by yhuai; I'm opening this PR to factor out this change from #7685, a larger pull request which contains two other optimizations. !-- Reviewable:start -- [img src=https://reviewable.io/review_button.png; height=40 alt=Review on Reviewable/](https://reviewable.io/reviews/apache/spark/7773) !-- Reviewable:end -- Author: Yin Huai yh...@databricks.com Author: Josh Rosen joshro...@databricks.com Closes #7773 from JoshRosen/multi-way-join-planning-improvements and squashes the following commits: 5c45924 [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements cd8269b [Josh Rosen] Refactor test to use SQLTestUtils 2963857 [Yin Huai] Revert unnecessary SqlConf change. 73913f7 [Yin Huai] Add comments and test. Also, revert the change in ShuffledHashOuterJoin for now. 4a99204 [Josh Rosen] Delete unrelated expression change 884ab95 [Josh Rosen] Carve out only SPARK-2205 changes. 247e5fa [Josh Rosen] Merge remote-tracking branch 'origin/master' into multi-way-join-planning-improvements c57a954 [Yin Huai] Bug fix. d3d2e64 [Yin Huai] First round of cleanup. f9516b0 [Yin Huai] Style c6667e7 [Yin Huai] Add PartitioningCollection. e616d3b [Yin Huai] wip 7c2d2d8 [Yin Huai] Bug fix and refactoring. 69bb072 [Yin Huai] Introduce NullSafeHashPartitioning and NullUnsafePartitioning. d5b84c3 [Yin Huai] Do not add unnessary filters. 2201129 [Yin Huai] Filter out rows that will not be joined in equal joins early. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/114ff926 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/114ff926 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/114ff926 Branch: refs/heads/master Commit: 114ff926fcd078697c279b5cf6173b515865 Parents: 30e8911 Author: Yin Huai yh...@databricks.com Authored: Sun Aug 2 20:44:23 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun Aug 2 20:44:23 2015 -0700 -- .../catalyst/plans/physical/partitioning.scala | 87 +--- .../spark/sql/catalyst/DistributionSuite.scala | 2 +- .../apache/spark/sql/execution/Exchange.scala | 2 +- .../joins/BroadcastHashOuterJoin.scala | 4 +- .../sql/execution/joins/HashOuterJoin.scala | 9 -- .../sql/execution/joins/LeftSemiJoinHash.scala | 6 +- .../sql/execution/joins/ShuffledHashJoin.scala | 7 +- .../execution/joins/ShuffledHashOuterJoin.scala | 10 ++- .../sql/execution/joins/SortMergeJoin.scala | 3 +- .../spark/sql/execution/PlannerSuite.scala | 49 ++- 10 files changed, 148 insertions(+), 31 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/114ff926/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 f4d1dba..ec659ce 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 @@ -60,8 +60,9 @@ case class ClusteredDistribution(clustering: Seq[Expression]) extends Distributi /** * Represents data where tuples have been ordered according to the `ordering` * [[Expression Expressions]]. This is a strictly stronger guarantee than - * [[ClusteredDistribution]] as an ordering will ensure that tuples that share the same value for - * the ordering expressions are contiguous and will never be split across partitions. + * [[ClusteredDistribution]] as an ordering will ensure that tuples that share the + * same value for the ordering expressions are contiguous and will never be split across + * partitions. */ case class OrderedDistribution(ordering: Seq[SortOrder]) extends Distribution { require( @@ -86,8 +87,12 @@ sealed trait Partitioning
spark git commit: [SPARK-7289] [SPARK-9949] [SQL] Backport SPARK-7289 and SPARK-9949 to branch 1.4
Repository: spark Updated Branches: refs/heads/branch-1.4 f7f2ac69d - c73498773 [SPARK-7289] [SPARK-9949] [SQL] Backport SPARK-7289 and SPARK-9949 to branch 1.4 The bug fixed by SPARK-7289 is a pretty serious one (Spark SQL generates wrong results). We should backport the fix to branch 1.4 (https://github.com/apache/spark/pull/6780). Also, we need to backport the fix of `TakeOrderedAndProject` as well (https://github.com/apache/spark/pull/8179). Author: Wenchen Fan cloud0...@outlook.com Author: Yin Huai yh...@databricks.com Closes #8252 from yhuai/backport7289And9949. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/c7349877 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/c7349877 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/c7349877 Branch: refs/heads/branch-1.4 Commit: c73498773952e675c9ec9c7b5acfc8c293ed6b51 Parents: f7f2ac6 Author: Wenchen Fan cloud0...@outlook.com Authored: Mon Aug 17 21:22:01 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Aug 17 21:22:01 2015 -0700 -- .../scala/org/apache/spark/sql/SQLContext.scala | 2 +- .../spark/sql/execution/SparkStrategies.scala | 8 +++-- .../spark/sql/execution/basicOperators.scala| 34 .../spark/sql/execution/PlannerSuite.scala | 20 .../org/apache/spark/sql/hive/HiveContext.scala | 2 +- 5 files changed, 56 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/c7349877/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala index bd3f690..6a7569d 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/SQLContext.scala @@ -827,7 +827,7 @@ class SQLContext(@transient val sparkContext: SparkContext) experimental.extraStrategies ++ ( DataSourceStrategy :: DDLStrategy :: - TakeOrdered :: + TakeOrderedAndProject :: HashAggregation :: LeftSemiJoin :: HashJoin :: http://git-wip-us.apache.org/repos/asf/spark/blob/c7349877/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 d0a1ad0..bf75cc2 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 @@ -205,10 +205,14 @@ private[sql] abstract class SparkStrategies extends QueryPlanner[SparkPlan] { protected lazy val singleRowRdd = sparkContext.parallelize(Seq(new GenericRow(Array[Any]()): Row), 1) - object TakeOrdered extends Strategy { + object TakeOrderedAndProject extends Strategy { def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match { case logical.Limit(IntegerLiteral(limit), logical.Sort(order, true, child)) = -execution.TakeOrdered(limit, order, planLater(child)) :: Nil +execution.TakeOrderedAndProject(limit, order, None, planLater(child)) :: Nil + case logical.Limit( + IntegerLiteral(limit), + logical.Project(projectList, logical.Sort(order, true, child))) = +execution.TakeOrderedAndProject(limit, order, Some(projectList), planLater(child)) :: Nil case _ = Nil } } http://git-wip-us.apache.org/repos/asf/spark/blob/c7349877/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 a30ade8..72b8e48 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 @@ -144,20 +144,35 @@ case class Limit(limit: Int, child: SparkPlan) /** * :: DeveloperApi :: - * Take the first limit elements as defined by the sortOrder. This is logically equivalent to - * having a [[Limit]] operator after a [[Sort]] operator. This could have been named TopK, but - * Spark's top operator does the opposite in ordering so we name it TakeOrdered to avoid confusion. + * Take the first limit elements as defined by the sortOrder, and do projection if needed. + * This is logically equivalent to having a [[Limit
spark git commit: [SPARK-10005] [SQL] Fixes schema merging for nested structs
Repository: spark Updated Branches: refs/heads/branch-1.5 e2c6ef810 - 90245f65c [SPARK-10005] [SQL] Fixes schema merging for nested structs In case of schema merging, we only handled first level fields when converting Parquet groups to `InternalRow`s. Nested struct fields are not properly handled. For example, the schema of a Parquet file to be read can be: ``` message individual { required group f1 { optional binary f11 (utf8); } } ``` while the global schema is: ``` message global { required group f1 { optional binary f11 (utf8); optional int32 f12; } } ``` This PR fixes this issue by padding missing fields when creating actual converters. Author: Cheng Lian l...@databricks.com Closes #8228 from liancheng/spark-10005/nested-schema-merging. (cherry picked from commit ae2370e72f93db8a28b262e8252c55fe1fc9873c) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/90245f65 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/90245f65 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/90245f65 Branch: refs/heads/branch-1.5 Commit: 90245f65c94a40d3210207abaf6f136f5ce2861f Parents: e2c6ef8 Author: Cheng Lian l...@databricks.com Authored: Sun Aug 16 10:17:58 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun Aug 16 10:18:08 2015 -0700 -- .../parquet/CatalystReadSupport.scala | 19 -- .../parquet/CatalystRowConverter.scala | 70 ++-- .../parquet/CatalystSchemaConverter.scala | 15 + .../datasources/parquet/ParquetQuerySuite.scala | 30 - 4 files changed, 112 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/90245f65/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index 4049795..a4679bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.StructType private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with Logging { + // Called after `init()` when initializing Parquet record reader. override def prepareForRead( conf: Configuration, keyValueMetaData: JMap[String, String], @@ -51,19 +52,30 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with // available if the target file is written by Spark SQL. .orElse(metadata.get(CatalystReadSupport.SPARK_METADATA_KEY)) }.map(StructType.fromString).getOrElse { -logDebug(Catalyst schema not available, falling back to Parquet schema) +logInfo(Catalyst schema not available, falling back to Parquet schema) toCatalyst.convert(parquetRequestedSchema) } -logDebug(sCatalyst schema used to read Parquet files: $catalystRequestedSchema) +logInfo { + sGoing to read the following fields from the Parquet file: + | + |Parquet form: + |$parquetRequestedSchema + | + |Catalyst form: + |$catalystRequestedSchema + .stripMargin +} + new CatalystRecordMaterializer(parquetRequestedSchema, catalystRequestedSchema) } + // Called before `prepareForRead()` when initializing Parquet record reader. override def init(context: InitContext): ReadContext = { val conf = context.getConfiguration // If the target file was written by Spark SQL, we should be able to find a serialized Catalyst -// schema of this file from its the metadata. +// schema of this file from its metadata. val maybeRowSchema = Option(conf.get(RowWriteSupport.SPARK_ROW_SCHEMA)) // Optional schema of requested columns, in the form of a string serialized from a Catalyst @@ -141,7 +153,6 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with maybeRequestedSchema.map(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA - _) ++ maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA - _) -logInfo(sGoing to read Parquet file with these requested columns: $parquetRequestedSchema) new ReadContext(parquetRequestedSchema, metadata) } }
spark git commit: [SPARK-10005] [SQL] Fixes schema merging for nested structs
Repository: spark Updated Branches: refs/heads/master cf016075a - ae2370e72 [SPARK-10005] [SQL] Fixes schema merging for nested structs In case of schema merging, we only handled first level fields when converting Parquet groups to `InternalRow`s. Nested struct fields are not properly handled. For example, the schema of a Parquet file to be read can be: ``` message individual { required group f1 { optional binary f11 (utf8); } } ``` while the global schema is: ``` message global { required group f1 { optional binary f11 (utf8); optional int32 f12; } } ``` This PR fixes this issue by padding missing fields when creating actual converters. Author: Cheng Lian l...@databricks.com Closes #8228 from liancheng/spark-10005/nested-schema-merging. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ae2370e7 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ae2370e7 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ae2370e7 Branch: refs/heads/master Commit: ae2370e72f93db8a28b262e8252c55fe1fc9873c Parents: cf01607 Author: Cheng Lian l...@databricks.com Authored: Sun Aug 16 10:17:58 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Sun Aug 16 10:17:58 2015 -0700 -- .../parquet/CatalystReadSupport.scala | 19 -- .../parquet/CatalystRowConverter.scala | 70 ++-- .../parquet/CatalystSchemaConverter.scala | 15 + .../datasources/parquet/ParquetQuerySuite.scala | 30 - 4 files changed, 112 insertions(+), 22 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ae2370e7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala index 4049795..a4679bb 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystReadSupport.scala @@ -32,6 +32,7 @@ import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.types.StructType private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with Logging { + // Called after `init()` when initializing Parquet record reader. override def prepareForRead( conf: Configuration, keyValueMetaData: JMap[String, String], @@ -51,19 +52,30 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with // available if the target file is written by Spark SQL. .orElse(metadata.get(CatalystReadSupport.SPARK_METADATA_KEY)) }.map(StructType.fromString).getOrElse { -logDebug(Catalyst schema not available, falling back to Parquet schema) +logInfo(Catalyst schema not available, falling back to Parquet schema) toCatalyst.convert(parquetRequestedSchema) } -logDebug(sCatalyst schema used to read Parquet files: $catalystRequestedSchema) +logInfo { + sGoing to read the following fields from the Parquet file: + | + |Parquet form: + |$parquetRequestedSchema + | + |Catalyst form: + |$catalystRequestedSchema + .stripMargin +} + new CatalystRecordMaterializer(parquetRequestedSchema, catalystRequestedSchema) } + // Called before `prepareForRead()` when initializing Parquet record reader. override def init(context: InitContext): ReadContext = { val conf = context.getConfiguration // If the target file was written by Spark SQL, we should be able to find a serialized Catalyst -// schema of this file from its the metadata. +// schema of this file from its metadata. val maybeRowSchema = Option(conf.get(RowWriteSupport.SPARK_ROW_SCHEMA)) // Optional schema of requested columns, in the form of a string serialized from a Catalyst @@ -141,7 +153,6 @@ private[parquet] class CatalystReadSupport extends ReadSupport[InternalRow] with maybeRequestedSchema.map(CatalystReadSupport.SPARK_ROW_REQUESTED_SCHEMA - _) ++ maybeRowSchema.map(RowWriteSupport.SPARK_ROW_SCHEMA - _) -logInfo(sGoing to read Parquet file with these requested columns: $parquetRequestedSchema) new ReadContext(parquetRequestedSchema, metadata) } } http://git-wip-us.apache.org/repos/asf/spark/blob/ae2370e7/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/CatalystRowConverter.scala
spark git commit: [SPARK-10143] [SQL] Use parquet's block size (row group size) setting as the min split size if necessary.
Repository: spark Updated Branches: refs/heads/master f5b028ed2 - e3355090d [SPARK-10143] [SQL] Use parquet's block size (row group size) setting as the min split size if necessary. https://issues.apache.org/jira/browse/SPARK-10143 With this PR, we will set min split size to parquet's block size (row group size) set in the conf if the min split size is smaller. So, we can avoid have too many tasks and even useless tasks for reading parquet data. I tested it locally. The table I have has 343MB and it is in my local FS. Because I did not set any min/max split size, the default split size was 32MB and the map stage had 11 tasks. But there were only three tasks that actually read data. With my PR, there were only three tasks in the map stage. Here is the difference. Without this PR: ![image](https://cloud.githubusercontent.com/assets/2072857/9399179/8587dba6-4765-11e5-9189-7ebba52a2b6d.png) With this PR: ![image](https://cloud.githubusercontent.com/assets/2072857/9399185/a4735d74-4765-11e5-8848-1f1e361a6b4b.png) Even if the block size setting does match the actual block size of parquet file, I think it is still generally good to use parquet's block size setting if min split size is smaller than this block size. Tested it on a cluster using ``` val count = sqlContext.table(store_sales).groupBy().count().queryExecution.executedPlan(3).execute().count ``` Basically, it reads 0 column of table `store_sales`. My table has 1824 parquet files with size from 80MB to 280MB (1 to 3 row group sizes). Without this patch, in a 16 worker cluster, the job had 5023 tasks and spent 102s. With this patch, the job had 2893 tasks and spent 64s. It is still not as good as using one mapper per file (1824 tasks and 42s), but it is much better than our master. Author: Yin Huai yh...@databricks.com Closes #8346 from yhuai/parquetMinSplit. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e3355090 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e3355090 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e3355090 Branch: refs/heads/master Commit: e3355090d4030daffed5efb0959bf1d724c13c13 Parents: f5b028e Author: Yin Huai yh...@databricks.com Authored: Fri Aug 21 14:30:00 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Aug 21 14:30:00 2015 -0700 -- .../datasources/parquet/ParquetRelation.scala | 41 +++- 1 file changed, 39 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e3355090/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 68169d4..bbf682a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -26,6 +26,7 @@ import scala.collection.mutable import scala.util.{Failure, Try} import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -281,12 +282,18 @@ private[sql] class ParquetRelation( val assumeInt96IsTimestamp = sqlContext.conf.isParquetINT96AsTimestamp val followParquetFormatSpec = sqlContext.conf.followParquetFormatSpec +// Parquet row group size. We will use this value as the value for +// mapreduce.input.fileinputformat.split.minsize and mapred.min.split.size if the value +// of these flags are smaller than the parquet row group size. +val parquetBlockSize = ParquetOutputFormat.getLongBlockSize(broadcastedConf.value.value) + // Create the function to set variable Parquet confs at both driver and executor side. val initLocalJobFuncOpt = ParquetRelation.initializeLocalJobFunc( requiredColumns, filters, dataSchema, +parquetBlockSize, useMetadataCache, parquetFilterPushDown, assumeBinaryIsString, @@ -294,7 +301,8 @@ private[sql] class ParquetRelation( followParquetFormatSpec) _ // Create the function to set input paths at the driver side. -val setInputPaths = ParquetRelation.initializeDriverSideJobFunc(inputFiles) _ +val setInputPaths = + ParquetRelation.initializeDriverSideJobFunc(inputFiles, parquetBlockSize) _ Utils.withDummyCallSite(sqlContext.sparkContext) { new SqlNewHadoopRDD( @@ -482,11 +490,35 @@ private[sql
spark git commit: [SPARK-10143] [SQL] Use parquet's block size (row group size) setting as the min split size if necessary.
Repository: spark Updated Branches: refs/heads/branch-1.5 e7db8761b - 14c8c0c0d [SPARK-10143] [SQL] Use parquet's block size (row group size) setting as the min split size if necessary. https://issues.apache.org/jira/browse/SPARK-10143 With this PR, we will set min split size to parquet's block size (row group size) set in the conf if the min split size is smaller. So, we can avoid have too many tasks and even useless tasks for reading parquet data. I tested it locally. The table I have has 343MB and it is in my local FS. Because I did not set any min/max split size, the default split size was 32MB and the map stage had 11 tasks. But there were only three tasks that actually read data. With my PR, there were only three tasks in the map stage. Here is the difference. Without this PR: ![image](https://cloud.githubusercontent.com/assets/2072857/9399179/8587dba6-4765-11e5-9189-7ebba52a2b6d.png) With this PR: ![image](https://cloud.githubusercontent.com/assets/2072857/9399185/a4735d74-4765-11e5-8848-1f1e361a6b4b.png) Even if the block size setting does match the actual block size of parquet file, I think it is still generally good to use parquet's block size setting if min split size is smaller than this block size. Tested it on a cluster using ``` val count = sqlContext.table(store_sales).groupBy().count().queryExecution.executedPlan(3).execute().count ``` Basically, it reads 0 column of table `store_sales`. My table has 1824 parquet files with size from 80MB to 280MB (1 to 3 row group sizes). Without this patch, in a 16 worker cluster, the job had 5023 tasks and spent 102s. With this patch, the job had 2893 tasks and spent 64s. It is still not as good as using one mapper per file (1824 tasks and 42s), but it is much better than our master. Author: Yin Huai yh...@databricks.com Closes #8346 from yhuai/parquetMinSplit. (cherry picked from commit e3355090d4030daffed5efb0959bf1d724c13c13) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/14c8c0c0 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/14c8c0c0 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/14c8c0c0 Branch: refs/heads/branch-1.5 Commit: 14c8c0c0da1184c587f0d5ab60f1d56feaa588e4 Parents: e7db876 Author: Yin Huai yh...@databricks.com Authored: Fri Aug 21 14:30:00 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Aug 21 14:30:12 2015 -0700 -- .../datasources/parquet/ParquetRelation.scala | 41 +++- 1 file changed, 39 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/14c8c0c0/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala index 68169d4..bbf682a 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/parquet/ParquetRelation.scala @@ -26,6 +26,7 @@ import scala.collection.mutable import scala.util.{Failure, Try} import com.google.common.base.Objects +import org.apache.hadoop.conf.Configuration import org.apache.hadoop.fs.{FileStatus, Path} import org.apache.hadoop.io.Writable import org.apache.hadoop.mapreduce._ @@ -281,12 +282,18 @@ private[sql] class ParquetRelation( val assumeInt96IsTimestamp = sqlContext.conf.isParquetINT96AsTimestamp val followParquetFormatSpec = sqlContext.conf.followParquetFormatSpec +// Parquet row group size. We will use this value as the value for +// mapreduce.input.fileinputformat.split.minsize and mapred.min.split.size if the value +// of these flags are smaller than the parquet row group size. +val parquetBlockSize = ParquetOutputFormat.getLongBlockSize(broadcastedConf.value.value) + // Create the function to set variable Parquet confs at both driver and executor side. val initLocalJobFuncOpt = ParquetRelation.initializeLocalJobFunc( requiredColumns, filters, dataSchema, +parquetBlockSize, useMetadataCache, parquetFilterPushDown, assumeBinaryIsString, @@ -294,7 +301,8 @@ private[sql] class ParquetRelation( followParquetFormatSpec) _ // Create the function to set input paths at the driver side. -val setInputPaths = ParquetRelation.initializeDriverSideJobFunc(inputFiles) _ +val setInputPaths = + ParquetRelation.initializeDriverSideJobFunc(inputFiles, parquetBlockSize
spark git commit: [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key in aggregation.
Repository: spark Updated Branches: refs/heads/branch-1.5 675e22494 - 5be517584 [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key in aggregation. This improves performance by ~ 20 - 30% in one of my local test and should fix the performance regression from 1.4 to 1.5 on ss_max. Author: Reynold Xin r...@databricks.com Closes #8332 from rxin/SPARK-10100. (cherry picked from commit b4f4e91c395cb69ced61d9ff1492d1b814f96828) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/5be51758 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/5be51758 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/5be51758 Branch: refs/heads/branch-1.5 Commit: 5be517584be0c78dc4641a4aa14ea9da05ed344d Parents: 675e224 Author: Reynold Xin r...@databricks.com Authored: Thu Aug 20 07:53:27 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu Aug 20 07:53:40 2015 -0700 -- .../execution/aggregate/TungstenAggregate.scala | 2 +- .../aggregate/TungstenAggregationIterator.scala | 30 ++-- 2 files changed, 22 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/5be51758/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 99f51ba..ba379d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -104,7 +104,7 @@ case class TungstenAggregate( } else { // This is a grouped aggregate and the input iterator is empty, // so return an empty iterator. - Iterator[UnsafeRow]() + Iterator.empty } } else { aggregationIterator.start(parentIterator) http://git-wip-us.apache.org/repos/asf/spark/blob/5be51758/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index af7e0fc..26fdbc8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -357,18 +357,30 @@ class TungstenAggregationIterator( // sort-based aggregation (by calling switchToSortBasedAggregation). private def processInputs(): Unit = { assert(inputIter != null, attempted to process input when iterator was null) -while (!sortBased inputIter.hasNext) { - val newInput = inputIter.next() - numInputRows += 1 - val groupingKey = groupProjection.apply(newInput) +if (groupingExpressions.isEmpty) { + // If there is no grouping expressions, we can just reuse the same buffer over and over again. + // Note that it would be better to eliminate the hash map entirely in the future. + val groupingKey = groupProjection.apply(null) val buffer: UnsafeRow = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) - if (buffer == null) { -// buffer == null means that we could not allocate more memory. -// Now, we need to spill the map and switch to sort-based aggregation. -switchToSortBasedAggregation(groupingKey, newInput) - } else { + while (inputIter.hasNext) { +val newInput = inputIter.next() +numInputRows += 1 processRow(buffer, newInput) } +} else { + while (!sortBased inputIter.hasNext) { +val newInput = inputIter.next() +numInputRows += 1 +val groupingKey = groupProjection.apply(newInput) +val buffer: UnsafeRow = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) +if (buffer == null) { + // buffer == null means that we could not allocate more memory. + // Now, we need to spill the map and switch to sort-based aggregation. + switchToSortBasedAggregation(groupingKey, newInput) +} else { + processRow(buffer, newInput) +} + } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail:
spark git commit: [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key in aggregation.
Repository: spark Updated Branches: refs/heads/master 43e013542 - b4f4e91c3 [SPARK-10100] [SQL] Eliminate hash table lookup if there is no grouping key in aggregation. This improves performance by ~ 20 - 30% in one of my local test and should fix the performance regression from 1.4 to 1.5 on ss_max. Author: Reynold Xin r...@databricks.com Closes #8332 from rxin/SPARK-10100. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/b4f4e91c Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/b4f4e91c Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/b4f4e91c Branch: refs/heads/master Commit: b4f4e91c395cb69ced61d9ff1492d1b814f96828 Parents: 43e0135 Author: Reynold Xin r...@databricks.com Authored: Thu Aug 20 07:53:27 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Thu Aug 20 07:53:27 2015 -0700 -- .../execution/aggregate/TungstenAggregate.scala | 2 +- .../aggregate/TungstenAggregationIterator.scala | 30 ++-- 2 files changed, 22 insertions(+), 10 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/b4f4e91c/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala index 99f51ba..ba379d3 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala @@ -104,7 +104,7 @@ case class TungstenAggregate( } else { // This is a grouped aggregate and the input iterator is empty, // so return an empty iterator. - Iterator[UnsafeRow]() + Iterator.empty } } else { aggregationIterator.start(parentIterator) http://git-wip-us.apache.org/repos/asf/spark/blob/b4f4e91c/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala index af7e0fc..26fdbc8 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregationIterator.scala @@ -357,18 +357,30 @@ class TungstenAggregationIterator( // sort-based aggregation (by calling switchToSortBasedAggregation). private def processInputs(): Unit = { assert(inputIter != null, attempted to process input when iterator was null) -while (!sortBased inputIter.hasNext) { - val newInput = inputIter.next() - numInputRows += 1 - val groupingKey = groupProjection.apply(newInput) +if (groupingExpressions.isEmpty) { + // If there is no grouping expressions, we can just reuse the same buffer over and over again. + // Note that it would be better to eliminate the hash map entirely in the future. + val groupingKey = groupProjection.apply(null) val buffer: UnsafeRow = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) - if (buffer == null) { -// buffer == null means that we could not allocate more memory. -// Now, we need to spill the map and switch to sort-based aggregation. -switchToSortBasedAggregation(groupingKey, newInput) - } else { + while (inputIter.hasNext) { +val newInput = inputIter.next() +numInputRows += 1 processRow(buffer, newInput) } +} else { + while (!sortBased inputIter.hasNext) { +val newInput = inputIter.next() +numInputRows += 1 +val groupingKey = groupProjection.apply(newInput) +val buffer: UnsafeRow = hashMap.getAggregationBufferFromUnsafeRow(groupingKey) +if (buffer == null) { + // buffer == null means that we could not allocate more memory. + // Now, we need to spill the map and switch to sort-based aggregation. + switchToSortBasedAggregation(groupingKey, newInput) +} else { + processRow(buffer, newInput) +} + } } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8567] [SQL] Add logs to record the progress of HiveSparkSubmitSuite (1.4 branch)
Repository: spark Updated Branches: refs/heads/branch-1.4 187015f67 - 0de1737a8 [SPARK-8567] [SQL] Add logs to record the progress of HiveSparkSubmitSuite (1.4 branch) Cherry-pick f9b397f54d1c491680d70aba210bb8211fd249c1 to branch 1.4. Author: Yin Huai yh...@databricks.com Closes #7092 from yhuai/SPARK-8567-1.4 and squashes the following commits: 0ae2e14 [Yin Huai] [SPARK-8567] [SQL] Add logs to record the progress of HiveSparkSubmitSuite. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/0de1737a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/0de1737a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/0de1737a Branch: refs/heads/branch-1.4 Commit: 0de1737a8adf458aa578cf18b1bef1eb618c5783 Parents: 187015f Author: Yin Huai yh...@databricks.com Authored: Mon Jun 29 15:20:35 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 29 15:20:35 2015 -0700 -- .../org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala| 9 + 1 file changed, 9 insertions(+) -- http://git-wip-us.apache.org/repos/asf/spark/blob/0de1737a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala -- diff --git a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala index 8ca7a80..dde0d9e 100644 --- a/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala +++ b/sql/hive/src/test/scala/org/apache/spark/sql/hive/HiveSparkSubmitSuite.scala @@ -107,6 +107,7 @@ object SparkSubmitClassLoaderTest extends Logging { val sc = new SparkContext(conf) val hiveContext = new TestHiveContext(sc) val df = hiveContext.createDataFrame((1 to 100).map(i = (i, i))).toDF(i, j) +logInfo(Testing load classes at the driver side.) // First, we load classes at driver side. try { Class.forName(args(0), true, Thread.currentThread().getContextClassLoader) @@ -116,6 +117,7 @@ object SparkSubmitClassLoaderTest extends Logging { throw new Exception(Could not load user class from jar:\n, t) } // Second, we load classes at the executor side. +logInfo(Testing load classes at the executor side.) val result = df.mapPartitions { x = var exception: String = null try { @@ -133,6 +135,7 @@ object SparkSubmitClassLoaderTest extends Logging { } // Load a Hive UDF from the jar. +logInfo(Registering temporary Hive UDF provided in a jar.) hiveContext.sql( |CREATE TEMPORARY FUNCTION example_max @@ -142,18 +145,23 @@ object SparkSubmitClassLoaderTest extends Logging { hiveContext.createDataFrame((1 to 10).map(i = (i, sstr$i))).toDF(key, val) source.registerTempTable(sourceTable) // Load a Hive SerDe from the jar. +logInfo(Creating a Hive table with a SerDe provided in a jar.) hiveContext.sql( |CREATE TABLE t1(key int, val string) |ROW FORMAT SERDE 'org.apache.hive.hcatalog.data.JsonSerDe' .stripMargin) // Actually use the loaded UDF and SerDe. +logInfo(Writing data into the table.) hiveContext.sql( INSERT INTO TABLE t1 SELECT example_max(key) as key, val FROM sourceTable GROUP BY val) +logInfo(Running a simple query on the table.) val count = hiveContext.table(t1).orderBy(key, val).count() if (count != 10) { throw new Exception(stable t1 should have 10 rows instead of $count rows) } +logInfo(Test finishes.) +sc.stop() } } @@ -191,5 +199,6 @@ object SparkSQLConfTest extends Logging { val hiveContext = new TestHiveContext(sc) // Run a simple command to make sure all lazy vals in hiveContext get instantiated. hiveContext.tables().collect() +sc.stop() } } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8715] ArrayOutOfBoundsException fixed for DataFrameStatSuite.crosstab
Repository: spark Updated Branches: refs/heads/branch-1.4 80d53565a - ffc793a6c [SPARK-8715] ArrayOutOfBoundsException fixed for DataFrameStatSuite.crosstab cc yhuai Author: Burak Yavuz brk...@gmail.com Closes #7100 from brkyvz/ct-flakiness-fix and squashes the following commits: abc299a [Burak Yavuz] change 'to' to until 7e96d7c [Burak Yavuz] ArrayOutOfBoundsException fixed for DataFrameStatSuite.crosstab (cherry picked from commit ecacb1e88a135c802e253793e7c863d6ca8d2408) Signed-off-by: Yin Huai yh...@databricks.com Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/ffc793a6 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/ffc793a6 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/ffc793a6 Branch: refs/heads/branch-1.4 Commit: ffc793a6caaf8e6190651af4ff2d46492f062fc0 Parents: 80d5356 Author: Burak Yavuz brk...@gmail.com Authored: Mon Jun 29 18:48:28 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 29 18:48:38 2015 -0700 -- .../src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/ffc793a6/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala -- diff --git a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala index f740fb2..3e87eba 100644 --- a/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala +++ b/sql/core/src/test/scala/org/apache/spark/sql/DataFrameStatSuite.scala @@ -78,7 +78,7 @@ class DataFrameStatSuite extends SparkFunSuite { val rows = crosstab.collect() rows.foreach { row = val i = row.getString(0).toInt - for (col - 1 to 9) { + for (col - 1 until columnNames.length) { val j = columnNames(col).toInt assert(row.getLong(col) === expected.getOrElse((i, j), 0).toLong) } - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8650] [SQL] Use the user-specified app name priority in SparkSQLCLIDriver or HiveThriftServer2
Repository: spark Updated Branches: refs/heads/master f79410c49 - e6c3f7462 [SPARK-8650] [SQL] Use the user-specified app name priority in SparkSQLCLIDriver or HiveThriftServer2 When run `./bin/spark-sql --name query1.sql` [Before] ![before](https://cloud.githubusercontent.com/assets/1400819/8370336/fa20b75a-1bf8-11e5-9171-040049a53240.png) [After] ![after](https://cloud.githubusercontent.com/assets/1400819/8370189/dcc35cb4-1bf6-11e5-8796-a0694140bffb.png) Author: Yadong Qi qiyadong2...@gmail.com Closes #7030 from watermen/SPARK-8650 and squashes the following commits: 51b5134 [Yadong Qi] Improve code and add comment. e3d7647 [Yadong Qi] use spark.app.name priority. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/e6c3f746 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/e6c3f746 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/e6c3f746 Branch: refs/heads/master Commit: e6c3f7462b3fde220ec0084b52388dd4dabb75b9 Parents: f79410c Author: Yadong Qi qiyadong2...@gmail.com Authored: Mon Jun 29 22:34:38 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Mon Jun 29 22:34:38 2015 -0700 -- .../org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala | 7 ++- 1 file changed, 6 insertions(+), 1 deletion(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/e6c3f746/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala -- diff --git a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala index 79eda1f..1d41c46 100644 --- a/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala +++ b/sql/hive-thriftserver/src/main/scala/org/apache/spark/sql/hive/thriftserver/SparkSQLEnv.scala @@ -38,9 +38,14 @@ private[hive] object SparkSQLEnv extends Logging { val sparkConf = new SparkConf(loadDefaults = true) val maybeSerializer = sparkConf.getOption(spark.serializer) val maybeKryoReferenceTracking = sparkConf.getOption(spark.kryo.referenceTracking) + // If user doesn't specify the appName, we want to get [SparkSQL::localHostName] instead of + // the default appName [SparkSQLCLIDriver] in cli or beeline. + val maybeAppName = sparkConf +.getOption(spark.app.name) +.filterNot(_ == classOf[SparkSQLCLIDriver].getName) sparkConf -.setAppName(sSparkSQL::${Utils.localHostName()}) + .setAppName(maybeAppName.getOrElse(sSparkSQL::${Utils.localHostName()})) .set( spark.serializer, maybeSerializer.getOrElse(org.apache.spark.serializer.KryoSerializer)) - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-9422] [SQL] Remove the placeholder attributes used in the aggregation buffers
Repository: spark Updated Branches: refs/heads/master e78ec1a8f - 3744b7fd4 [SPARK-9422] [SQL] Remove the placeholder attributes used in the aggregation buffers https://issues.apache.org/jira/browse/SPARK-9422 Author: Yin Huai yh...@databricks.com Closes #7737 from yhuai/removePlaceHolder and squashes the following commits: ec29b44 [Yin Huai] Remove placeholder attributes. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3744b7fd Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3744b7fd Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3744b7fd Branch: refs/heads/master Commit: 3744b7fd42e52011af60cc205fcb4e4b23b35c68 Parents: e78ec1a Author: Yin Huai yh...@databricks.com Authored: Tue Jul 28 19:01:25 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Tue Jul 28 19:01:25 2015 -0700 -- .../expressions/aggregate/interfaces.scala | 27 ++- .../aggregate/aggregateOperators.scala | 4 +- .../aggregate/sortBasedIterators.scala | 209 +++ .../spark/sql/execution/aggregate/udaf.scala| 17 +- .../spark/sql/execution/aggregate/utils.scala | 4 +- 5 files changed, 121 insertions(+), 140 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3744b7fd/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala -- diff --git a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala index 10bd19c..9fb7623 100644 --- a/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala +++ b/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/interfaces.scala @@ -103,9 +103,30 @@ abstract class AggregateFunction2 final override def foldable: Boolean = false /** - * The offset of this function's buffer in the underlying buffer shared with other functions. + * The offset of this function's start buffer value in the + * underlying shared mutable aggregation buffer. + * For example, we have two aggregate functions `avg(x)` and `avg(y)`, which share + * the same aggregation buffer. In this shared buffer, the position of the first + * buffer value of `avg(x)` will be 0 and the position of the first buffer value of `avg(y)` + * will be 2. */ - var bufferOffset: Int = 0 + var mutableBufferOffset: Int = 0 + + /** + * The offset of this function's start buffer value in the + * underlying shared input aggregation buffer. An input aggregation buffer is used + * when we merge two aggregation buffers and it is basically the immutable one + * (we merge an input aggregation buffer and a mutable aggregation buffer and + * then store the new buffer values to the mutable aggregation buffer). + * Usually, an input aggregation buffer also contain extra elements like grouping + * keys at the beginning. So, mutableBufferOffset and inputBufferOffset are often + * different. + * For example, we have a grouping expression `key``, and two aggregate functions + * `avg(x)` and `avg(y)`. In this shared input aggregation buffer, the position of the first + * buffer value of `avg(x)` will be 1 and the position of the first buffer value of `avg(y)` + * will be 3 (position 0 is used for the value of key`). + */ + var inputBufferOffset: Int = 0 /** The schema of the aggregation buffer. */ def bufferSchema: StructType @@ -176,7 +197,7 @@ abstract class AlgebraicAggregate extends AggregateFunction2 with Serializable w override def initialize(buffer: MutableRow): Unit = { var i = 0 while (i bufferAttributes.size) { - buffer(i + bufferOffset) = initialValues(i).eval() + buffer(i + mutableBufferOffset) = initialValues(i).eval() i += 1 } } http://git-wip-us.apache.org/repos/asf/spark/blob/3744b7fd/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala -- diff --git a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala index 0c90828..98538c4 100644 --- a/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala +++ b/sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/aggregateOperators.scala @@ -72,8 +72,10 @@ case class Aggregate2Sort( protected override def doExecute(): RDD[InternalRow] = attachTree(this, execute) { child.execute().mapPartitions { iter
spark git commit: [SPARK-9466] [SQL] Increate two timeouts in CliSuite.
Repository: spark Updated Branches: refs/heads/master fbef566a1 - 815c8245f [SPARK-9466] [SQL] Increate two timeouts in CliSuite. Hopefully this can resolve the flakiness of this suite. JIRA: https://issues.apache.org/jira/browse/SPARK-9466 Author: Yin Huai yh...@databricks.com Closes # from yhuai/SPARK-9466 and squashes the following commits: e0e3a86 [Yin Huai] Increate the timeout. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/815c8245 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/815c8245 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/815c8245 Branch: refs/heads/master Commit: 815c8245f47e61226a04e2e02f508457b5e9e536 Parents: fbef566 Author: Yin Huai yh...@databricks.com Authored: Fri Jul 31 13:45:12 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jul 31 13:45:12 2015 -0700 -- .../scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/815c8245/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala -- diff --git a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala index 13b0c59..df80d04 100644 --- a/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala +++ b/sql/hive-thriftserver/src/test/scala/org/apache/spark/sql/hive/thriftserver/CliSuite.scala @@ -137,7 +137,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { } test(Single command with --database) { -runCliWithin(1.minute)( +runCliWithin(2.minute)( CREATE DATABASE hive_test_db; - OK, USE hive_test_db; @@ -148,7 +148,7 @@ class CliSuite extends SparkFunSuite with BeforeAndAfter with Logging { - Time taken: ) -runCliWithin(1.minute, Seq(--database, hive_test_db, -e, SHOW TABLES;))( +runCliWithin(2.minute, Seq(--database, hive_test_db, -e, SHOW TABLES;))( - OK, - To unsubscribe, e-mail: commits-unsubscr...@spark.apache.org For additional commands, e-mail: commits-h...@spark.apache.org
spark git commit: [SPARK-8640] [SQL] Enable Processing of Multiple Window Frames in a Single Window Operator
Repository: spark Updated Branches: refs/heads/master 0a1d2ca42 - 39ab199a3 [SPARK-8640] [SQL] Enable Processing of Multiple Window Frames in a Single Window Operator This PR enables the processing of multiple window frames in a single window operator. This should improve the performance of processing multiple window expressions wich share partition by/order by clauses, because it will be more efficient with respect to memory use and group processing. Author: Herman van Hovell hvanhov...@questtec.nl Closes #7515 from hvanhovell/SPARK-8640 and squashes the following commits: f0e1c21 [Herman van Hovell] Changed Window Logical/Physical plans to use partition by/order by specs directly instead of using WindowSpec. e1711c2 [Herman van Hovell] Enabled the processing of multiple window frames in a single Window operator. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/39ab199a Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/39ab199a Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/39ab199a Branch: refs/heads/master Commit: 39ab199a3f735b7658ab3331d3e2fb03441aec13 Parents: 0a1d2ca Author: Herman van Hovell hvanhov...@questtec.nl Authored: Fri Jul 31 12:07:18 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jul 31 12:08:25 2015 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 12 +++- .../catalyst/plans/logical/basicOperators.scala | 3 ++- .../spark/sql/execution/SparkStrategies.scala| 5 +++-- .../org/apache/spark/sql/execution/Window.scala | 19 ++- .../spark/sql/hive/execution/HivePlanTest.scala | 18 ++ 5 files changed, 40 insertions(+), 17 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/39ab199a/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 265f3d1..51d910b 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 @@ -347,7 +347,7 @@ class Analyzer( val newOutput = oldVersion.generatorOutput.map(_.newInstance()) (oldVersion, oldVersion.copy(generatorOutput = newOutput)) - case oldVersion @ Window(_, windowExpressions, _, child) + case oldVersion @ Window(_, windowExpressions, _, _, child) if AttributeSet(windowExpressions.map(_.toAttribute)).intersect(conflictingAttributes) .nonEmpty = (oldVersion, oldVersion.copy(windowExpressions = newAliases(windowExpressions))) @@ -825,7 +825,7 @@ class Analyzer( }.asInstanceOf[NamedExpression] } - // Second, we group extractedWindowExprBuffer based on their Window Spec. + // Second, we group extractedWindowExprBuffer based on their Partition and Order Specs. val groupedWindowExpressions = extractedWindowExprBuffer.groupBy { expr = val distinctWindowSpec = expr.collect { case window: WindowExpression = window.windowSpec @@ -841,7 +841,8 @@ class Analyzer( failAnalysis(s$expr has multiple Window Specifications ($distinctWindowSpec). + sPlease file a bug report with this error message, stack trace, and the query.) } else { - distinctWindowSpec.head + val spec = distinctWindowSpec.head + (spec.partitionSpec, spec.orderSpec) } }.toSeq @@ -850,9 +851,10 @@ class Analyzer( var currentChild = child var i = 0 while (i groupedWindowExpressions.size) { -val (windowSpec, windowExpressions) = groupedWindowExpressions(i) +val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i) // Set currentChild to the newly created Window operator. -currentChild = Window(currentChild.output, windowExpressions, windowSpec, currentChild) +currentChild = Window(currentChild.output, windowExpressions, + partitionSpec, orderSpec, currentChild) // Move to next Window Spec. i += 1 http://git-wip-us.apache.org/repos/asf/spark/blob/39ab199a/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
spark git commit: [SPARK-9233] [SQL] Enable code-gen in window function unit tests
Repository: spark Updated Branches: refs/heads/master 710c2b5dd - 3fc0cb920 [SPARK-9233] [SQL] Enable code-gen in window function unit tests Since code-gen is enabled by default, it is better to run window function tests with code-gen. https://issues.apache.org/jira/browse/SPARK-9233 Author: Yin Huai yh...@databricks.com Closes #7832 from yhuai/SPARK-9233 and squashes the following commits: 4e4e4cc [Yin Huai] style ca80e07 [Yin Huai] Test window function with codegen. Project: http://git-wip-us.apache.org/repos/asf/spark/repo Commit: http://git-wip-us.apache.org/repos/asf/spark/commit/3fc0cb92 Tree: http://git-wip-us.apache.org/repos/asf/spark/tree/3fc0cb92 Diff: http://git-wip-us.apache.org/repos/asf/spark/diff/3fc0cb92 Branch: refs/heads/master Commit: 3fc0cb92001798167a14c1377362a3335397dd4c Parents: 710c2b5 Author: Yin Huai yh...@databricks.com Authored: Fri Jul 31 14:13:06 2015 -0700 Committer: Yin Huai yh...@databricks.com Committed: Fri Jul 31 14:13:06 2015 -0700 -- .../spark/sql/catalyst/analysis/Analyzer.scala | 9 +++-- .../HiveWindowFunctionQuerySuite.scala | 38 +++- 2 files changed, 12 insertions(+), 35 deletions(-) -- http://git-wip-us.apache.org/repos/asf/spark/blob/3fc0cb92/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 51d910b..f5daba1 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 @@ -853,8 +853,13 @@ class Analyzer( while (i groupedWindowExpressions.size) { val ((partitionSpec, orderSpec), windowExpressions) = groupedWindowExpressions(i) // Set currentChild to the newly created Window operator. -currentChild = Window(currentChild.output, windowExpressions, - partitionSpec, orderSpec, currentChild) +currentChild = + Window( +currentChild.output, +windowExpressions, +partitionSpec, +orderSpec, +currentChild) // Move to next Window Spec. i += 1 http://git-wip-us.apache.org/repos/asf/spark/blob/3fc0cb92/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala -- diff --git a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala index 24a758f..92bb9e6 100644 --- a/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala +++ b/sql/hive/compatibility/src/test/scala/org/apache/spark/sql/hive/execution/HiveWindowFunctionQuerySuite.scala @@ -32,7 +32,7 @@ import org.apache.spark.util.Utils * for different tests and there are a few properties needed to let Hive generate golden * files, every `createQueryTest` calls should explicitly set `reset` to `false`. */ -abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with BeforeAndAfter { +class HiveWindowFunctionQuerySuite extends HiveComparisonTest with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault private val testTempDir = Utils.createTempDir() @@ -759,21 +759,7 @@ abstract class HiveWindowFunctionQueryBaseSuite extends HiveComparisonTest with .stripMargin, reset = false) } -class HiveWindowFunctionQueryWithoutCodeGenSuite extends HiveWindowFunctionQueryBaseSuite { - var originalCodegenEnabled: Boolean = _ - override def beforeAll(): Unit = { -super.beforeAll() -originalCodegenEnabled = conf.codegenEnabled -sql(set spark.sql.codegen=false) - } - - override def afterAll(): Unit = { -sql(sset spark.sql.codegen=$originalCodegenEnabled) -super.afterAll() - } -} - -abstract class HiveWindowFunctionQueryFileBaseSuite +class HiveWindowFunctionQueryFileSuite extends HiveCompatibilitySuite with BeforeAndAfter { private val originalTimeZone = TimeZone.getDefault private val originalLocale = Locale.getDefault @@ -789,11 +775,11 @@ abstract class HiveWindowFunctionQueryFileBaseSuite // The following settings are used for generating golden files with Hive. // We have to use kryo to correctly let Hive serialize plans with window functions. // This is used to generate golden files. -sql(set