[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136509156
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala
 ---
@@ -0,0 +1,134 @@
+/*
+ * 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.util.Properties
+
+import scala.language.existentials
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.common.FileUtils
+import org.apache.hadoop.hive.ql.plan.TableDesc
+import org.apache.hadoop.hive.serde.serdeConstants
+import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
+import org.apache.hadoop.mapred._
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.util.Utils
+
+/**
+ * Command for writing the results of `query` to file system.
+ *
+ * The syntax of using this command in SQL is:
+ * {{{
+ *   INSERT OVERWRITE [LOCAL] DIRECTORY
+ *   path
+ *   [ROW FORMAT row_format]
+ *   [STORED AS file_format]
+ *   SELECT ...
+ * }}}
+ *
+ * @param isLocal whether the path specified in `storage` is a local 
directory
+ * @param storage storage format used to describe how the query result is 
stored.
+ * @param query the logical plan representing data to write to
+ * @param overwrite whthere overwrites existing directory
+ */
+case class InsertIntoHiveDirCommand(
+isLocal: Boolean,
+storage: CatalogStorageFormat,
+query: LogicalPlan,
+overwrite: Boolean) extends SaveAsHiveFile {
+
+  override def children: Seq[LogicalPlan] = query :: Nil
+
+  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): 
Seq[Row] = {
+assert(children.length == 1)
+assert(storage.locationUri.nonEmpty)
+
+val Array(cols, types) = children.head.output.foldLeft(Array("", "")) 
{ case (r, a) =>
+  r(0) = r(0) + a.name + ","
+  r(1) = r(1) + a.dataType.catalogString + ":"
+  r
+}
+
+val properties = new Properties()
+properties.put("columns", cols.dropRight(1))
+properties.put("columns.types", types.dropRight(1))
+
+val sqlContext = sparkSession.sqlContext
+
+properties.put(serdeConstants.SERIALIZATION_LIB,
+  storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
+
+import scala.collection.JavaConverters._
+properties.putAll(storage.properties.asJava)
+
+var tableDesc = new TableDesc(
+  Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: 
InputFormat[_, _]]],
+  Utils.classForName(storage.outputFormat.get),
+  properties
+)
--- End diff --

If we use the schema of the query as the dummyTableSchema, do we still need 
to populate the properties by ourselves?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19090: [SPARK-21877][DEPLOY, WINDOWS] Handle quotes in Windows ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19090
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81305/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19090: [SPARK-21877][DEPLOY, WINDOWS] Handle quotes in Windows ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19090
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19090: [SPARK-21877][DEPLOY, WINDOWS] Handle quotes in Windows ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19090
  
**[Test build #81305 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81305/testReport)**
 for PR 19090 at commit 
[`26fc756`](https://github.com/apache/spark/commit/26fc75670750b4710a2e0d97642ec1b52d51ffbb).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19082: [SPARK-21870][SQL] Split aggregation code into sm...

2017-08-31 Thread rednaxelafx
Github user rednaxelafx commented on a diff in the pull request:

https://github.com/apache/spark/pull/19082#discussion_r136506452
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ---
@@ -244,6 +246,92 @@ case class HashAggregateExec(
 
   protected override val shouldStopRequired = false
 
+  // We assume a prefix has lower cases and a name has camel cases
+  private val variableName = "^[a-z]+_[a-zA-Z]+[0-9]*".r
+
+  // Returns true if a given name id belongs to this `CodegenContext`
+  private def isVariable(nameId: String): Boolean = nameId match {
+case variableName() => true
+case _ => false
+  }
+
+  // Extracts all the outer references for a given `aggExpr`. This result 
will be used to split
+  // aggregation into small functions.
+  private def getOuterReferences(
+  ctx: CodegenContext,
+  aggExpr: Expression,
+  subExprs: Map[Expression, SubExprEliminationState]): Set[(String, 
String)] = {
+val stack = mutable.Stack[Expression](aggExpr)
+val argSet = mutable.Set[(String, String)]()
+val addIfNotLiteral = (value: String, tpe: String) => {
--- End diff --

Hmm. Just a cosmetic style comment: I would have declared `addIfNotLiteral 
` with a `def` instead of making it a `scala.Function2[String, String, Unit]`.

BTW, can we add a comment to `val argSet` for what those two fields of the 
`Tuple2[String, String]` means? And then also make this `addIfNotLiteral` 
function take the arguments in the same order as the tuple.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19082: [SPARK-21870][SQL] Split aggregation code into sm...

2017-08-31 Thread rednaxelafx
Github user rednaxelafx commented on a diff in the pull request:

https://github.com/apache/spark/pull/19082#discussion_r136506046
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ---
@@ -244,6 +246,92 @@ case class HashAggregateExec(
 
   protected override val shouldStopRequired = false
 
+  // We assume a prefix has lower cases and a name has camel cases
+  private val variableName = "^[a-z]+_[a-zA-Z]+[0-9]*".r
--- End diff --

I know the regular expression is tempting, but there's actually a better 
way to do this along your idea, under the current framework.

I've got a piece of code sitting in my own workspace that checks for Java 
identifiers:
```scala
object CodegenContext {
  private val javaKeywords = Set(
"abstract", "assert", "boolean", "break", "byte", "case", "catch", 
"char", "class",
"const", "continue", "default", "do", "double", "else", "extends", 
"false", "final",
"finally", "float", "for", "goto", "if", "implements", "import", 
"instanceof", "int",
"interface", "long", "native", "new", "null", "package", "private", 
"protected", "public",
"return", "short", "static", "strictfp", "super", "switch", 
"synchronized", "this",
"throw", "throws", "transient", "true", "try", "void", "volatile", 
"while"
  )

  def isJavaIdentifier(str: String): Boolean = str match {
case null | "" => false
case _ => java.lang.Character.isJavaIdentifierStart(str.charAt(0)) &&
  (1 until str.length).forall(
i => 
java.lang.Character.isJavaIdentifierPart(str.charAt(i))) &&
  !javaKeywords.contains(str)
  }
}
```
Feel free to use it here if you'd like. This is the way 
`java.lang.Character.isJavaIdentifierStart()` and 
`java.lang.Character.isJavaIdentifierPart()` is supposed to be used anyway, 
nothing creative.

If you want to use it in a `case` like the way you're using the regular 
expression, just wrap the util above into an `unapply()`. But I'd say simply 
making `def isVariable(nameId: String) = 
CodegenContext.isJavaIdentifier(nameId)` is clean enough.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136508091
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala
 ---
@@ -0,0 +1,134 @@
+/*
+ * 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.util.Properties
+
+import scala.language.existentials
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.common.FileUtils
+import org.apache.hadoop.hive.ql.plan.TableDesc
+import org.apache.hadoop.hive.serde.serdeConstants
+import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
+import org.apache.hadoop.mapred._
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.util.Utils
+
+/**
+ * Command for writing the results of `query` to file system.
+ *
+ * The syntax of using this command in SQL is:
+ * {{{
+ *   INSERT OVERWRITE [LOCAL] DIRECTORY
+ *   path
+ *   [ROW FORMAT row_format]
+ *   [STORED AS file_format]
+ *   SELECT ...
+ * }}}
+ *
+ * @param isLocal whether the path specified in `storage` is a local 
directory
+ * @param storage storage format used to describe how the query result is 
stored.
+ * @param query the logical plan representing data to write to
+ * @param overwrite whthere overwrites existing directory
+ */
+case class InsertIntoHiveDirCommand(
+isLocal: Boolean,
+storage: CatalogStorageFormat,
+query: LogicalPlan,
+overwrite: Boolean) extends SaveAsHiveFile {
+
+  override def children: Seq[LogicalPlan] = query :: Nil
+
+  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): 
Seq[Row] = {
+assert(children.length == 1)
+assert(storage.locationUri.nonEmpty)
+
+val Array(cols, types) = children.head.output.foldLeft(Array("", "")) 
{ case (r, a) =>
+  r(0) = r(0) + a.name + ","
+  r(1) = r(1) + a.dataType.catalogString + ":"
+  r
+}
+
+val properties = new Properties()
+properties.put("columns", cols.dropRight(1))
+properties.put("columns.types", types.dropRight(1))
+
+val sqlContext = sparkSession.sqlContext
+
+properties.put(serdeConstants.SERIALIZATION_LIB,
+  storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
+
+import scala.collection.JavaConverters._
+properties.putAll(storage.properties.asJava)
+
+var tableDesc = new TableDesc(
--- End diff --

`var` -> `val`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136508055
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala
 ---
@@ -0,0 +1,134 @@
+/*
+ * 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.util.Properties
+
+import scala.language.existentials
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.common.FileUtils
+import org.apache.hadoop.hive.ql.plan.TableDesc
+import org.apache.hadoop.hive.serde.serdeConstants
+import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
+import org.apache.hadoop.mapred._
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.util.Utils
+
+/**
+ * Command for writing the results of `query` to file system.
+ *
+ * The syntax of using this command in SQL is:
+ * {{{
+ *   INSERT OVERWRITE [LOCAL] DIRECTORY
+ *   path
+ *   [ROW FORMAT row_format]
+ *   [STORED AS file_format]
+ *   SELECT ...
+ * }}}
+ *
+ * @param isLocal whether the path specified in `storage` is a local 
directory
+ * @param storage storage format used to describe how the query result is 
stored.
+ * @param query the logical plan representing data to write to
+ * @param overwrite whthere overwrites existing directory
+ */
+case class InsertIntoHiveDirCommand(
+isLocal: Boolean,
+storage: CatalogStorageFormat,
+query: LogicalPlan,
+overwrite: Boolean) extends SaveAsHiveFile {
+
+  override def children: Seq[LogicalPlan] = query :: Nil
+
+  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): 
Seq[Row] = {
+assert(children.length == 1)
+assert(storage.locationUri.nonEmpty)
+
+val Array(cols, types) = children.head.output.foldLeft(Array("", "")) 
{ case (r, a) =>
+  r(0) = r(0) + a.name + ","
+  r(1) = r(1) + a.dataType.catalogString + ":"
+  r
+}
+
+val properties = new Properties()
+properties.put("columns", cols.dropRight(1))
+properties.put("columns.types", types.dropRight(1))
+
+val sqlContext = sparkSession.sqlContext
+
+properties.put(serdeConstants.SERIALIZATION_LIB,
+  storage.serde.getOrElse(classOf[LazySimpleSerDe].getName))
+
+import scala.collection.JavaConverters._
+properties.putAll(storage.properties.asJava)
+
+var tableDesc = new TableDesc(
+  Utils.classForName(storage.inputFormat.get).asInstanceOf[Class[_ <: 
InputFormat[_, _]]],
+  Utils.classForName(storage.outputFormat.get),
+  properties
+)
--- End diff --

I am not 100% sure the above logics work well for all Hive versions and all 
the file formats. Another safer way is to use our existing way 
```Scala
val hiveTable = 
HiveClientImpl.toHiveTable(dummyCatalogTableWithUserSpecifiedStorage)
val tableDesc = new TableDesc(
  hiveTable.getInputFormatClass,
  hiveTable.getOutputFormatClass,
  hiveTable.getMetadata
)
```


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18697: [SPARK-16683][SQL] Repeated joins to same table can leak...

2017-08-31 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/18697
  
shouldn't we fix `ProjectExec.outputPartitioning`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18869
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81306/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18869
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18869
  
**[Test build #81306 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81306/testReport)**
 for PR 18869 at commit 
[`b64c9e6`](https://github.com/apache/spark/commit/b64c9e67dfb34668ba93089d84f932778c005d7b).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136506540
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/execution/InsertIntoHiveDirCommand.scala
 ---
@@ -0,0 +1,134 @@
+/*
+ * 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.util.Properties
+
+import scala.language.existentials
+
+import org.apache.hadoop.fs.{FileSystem, Path}
+import org.apache.hadoop.hive.common.FileUtils
+import org.apache.hadoop.hive.ql.plan.TableDesc
+import org.apache.hadoop.hive.serde.serdeConstants
+import org.apache.hadoop.hive.serde2.`lazy`.LazySimpleSerDe
+import org.apache.hadoop.mapred._
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogStorageFormat
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkPlan
+import org.apache.spark.util.Utils
+
+/**
+ * Command for writing the results of `query` to file system.
+ *
+ * The syntax of using this command in SQL is:
+ * {{{
+ *   INSERT OVERWRITE [LOCAL] DIRECTORY
+ *   path
+ *   [ROW FORMAT row_format]
+ *   [STORED AS file_format]
+ *   SELECT ...
+ * }}}
+ *
+ * @param isLocal whether the path specified in `storage` is a local 
directory
+ * @param storage storage format used to describe how the query result is 
stored.
+ * @param query the logical plan representing data to write to
+ * @param overwrite whthere overwrites existing directory
+ */
+case class InsertIntoHiveDirCommand(
+isLocal: Boolean,
+storage: CatalogStorageFormat,
+query: LogicalPlan,
+overwrite: Boolean) extends SaveAsHiveFile {
+
+  override def children: Seq[LogicalPlan] = query :: Nil
+
+  override def run(sparkSession: SparkSession, children: Seq[SparkPlan]): 
Seq[Row] = {
+assert(children.length == 1)
+assert(storage.locationUri.nonEmpty)
+
+val Array(cols, types) = children.head.output.foldLeft(Array("", "")) 
{ case (r, a) =>
+  r(0) = r(0) + a.name + ","
+  r(1) = r(1) + a.dataType.catalogString + ":"
+  r
+}
+
+val properties = new Properties()
+properties.put("columns", cols.dropRight(1))
+properties.put("columns.types", types.dropRight(1))
+
+val sqlContext = sparkSession.sqlContext
--- End diff --

Nit: remove it. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19102: [SPARK-21859][CORE] Fix SparkFiles.get failed on driver ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19102
  
Can one of the admins verify this patch?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19079: [SPARK-21859][CORE] Fix SparkFiles.get failed on driver ...

2017-08-31 Thread lgrcyanny
Github user lgrcyanny commented on the issue:

https://github.com/apache/spark/pull/19079
  
Hi @vanzin I have submit a PR based on master branch, please review it, 
thank you
https://github.com/apache/spark/pull/19102


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19102: [SPARK-21859][CORE] Fix SparkFiles.get failed on ...

2017-08-31 Thread lgrcyanny
GitHub user lgrcyanny opened a pull request:

https://github.com/apache/spark/pull/19102

[SPARK-21859][CORE] Fix SparkFiles.get failed on driver in yarn-cluster and 
yarn-client mode

## What changes were proposed in this pull request?
when use SparkFiles.get a file on driver in yarn-client or yarn-cluster, it 
will report file not found exception.
This exception only happens on driver, SparkFiles.get on executor works 
fine.
we can reproduce the bug as follows:
```scala
val conf = new SparkConf().setAppName("SparkFilesTest")
val sc = new SparkContext(conf)
def testOnDriver(fileName: String) = {
val file = new File(SparkFiles.get(fileName))
if (!file.exists()) {
println(s"$file not exist")
} else {
// print file content on driver
val content = Source.fromFile(file).getLines().mkString("\n")
println(s"File content: ${content}")
}
}
// the output will be file not exist
```

```python
conf = SparkConf().setAppName("test files")
sc = SparkContext(appName="spark files test")
def test_on_driver(filename):
file = SparkFiles.get(filename)
print("file path: {}".format(file))
if os.path.exists(file):
with open(file) as f:
lines = f.readlines()
print(lines)
else:
print("file doesn't exist")
run_command("ls .")
```
the output will be file not exist

## How was this patch tested?
tested in integration tests and manual tests
submit the demo case in yarn-cluster and yarn-client mode, and verify the 
test result

```
./bin/spark-submit --master yarn-cluster --files README.md --class 
"testing.SparkFilesTest" testing.jar
./bin/spark-submit --master yarn-client --files README.md --class 
"testing.SparkFilesTest" testing.jar
./bin/spark-submit --master yarn-cluster --files README.md test_get_files.py
./bin/spark-submit --master yarn-client --files README.md test_get_files.py
```


You can merge this pull request into a Git repository by running:

$ git pull https://github.com/lgrcyanny/spark fix-spark-yarn-files-master

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/19102.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #19102


commit f2c06ffe4649589d2d175fc3dc61e00170c20a94
Author: lgrcyanny 
Date:   2017-05-07T12:51:55Z

[SPARK-21859][CORE] Fix SparkFiles.get failed on driver in yarn-cluster and 
yarn-client mode

when use SparkFiles.get a file on driver in yarn-client or yarn-cluster, it 
will report file not found exception.
This exception only happens on driver, SparkFiles.get on executor works 
fine.
we can reproduce the bug as follows:
```scala
val conf = new SparkConf().setAppName("SparkFilesTest")
val sc = new SparkContext(conf)
def testOnDriver(fileName: String) = {
val file = new File(SparkFiles.get(fileName))
if (!file.exists()) {
println(s"$file not exist")
} else {
// print file content on driver
val content = Source.fromFile(file).getLines().mkString("\n")
println(s"File content: ${content}")
}
}
// the output will be file not exist
```

```python
conf = SparkConf().setAppName("test files")
sc = SparkContext(appName="spark files test")
def test_on_driver(filename):
file = SparkFiles.get(filename)
print("file path: {}".format(file))
if os.path.exists(file):
with open(file) as f:
lines = f.readlines()
print(lines)
else:
print("file doesn't exist")
run_command("ls .")
```
the output will be file not exist

tested in integration tests and manual tests
submit the demo case in yarn-cluster and yarn-client mode, and verify the 
test result

```
./bin/spark-submit --master yarn-cluster --files README.md --class 
"testing.SparkFilesTest" testing.jar
./bin/spark-submit --master yarn-client --files README.md --class 
"testing.SparkFilesTest" testing.jar
./bin/spark-submit --master yarn-cluster --files README.md test_get_files.py
./bin/spark-submit --master yarn-client --files README.md test_get_files.py
```




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional 

[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/18869
  
@gatorsmile Right. Isn't too verbose if we describe map is not supported in 
each description?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/18869
  
`map` is not supported, right?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136502487
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
@@ -534,4 +534,115 @@ class InsertIntoHiveTableSuite extends QueryTest with 
TestHiveSingleton with Bef
   }
 }
   }
+
+  test("insert overwrite to dir from hive metastore table") {
+withTempDir { dir =>
+  val path = dir.toURI.getPath
+
+  sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src 
where key < 10")
+
+  sql(
+s"""
+   |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
+   |STORED AS orc
+   |SELECT * FROM src where key < 10
+ """.stripMargin)
+
+  // use orc data source to check the data of path is right.
+  withTempView("orc_source") {
+sql(
+  s"""
+ |CREATE TEMPORARY VIEW orc_source
+ |USING org.apache.spark.sql.hive.orc
+ |OPTIONS (
+ |  PATH '${dir.getCanonicalPath}'
+ |)
+   """.stripMargin)
+
+checkAnswer(
+  sql("select * from orc_source"),
+  sql("select * from src where key < 10").collect())
+  }
+}
+  }
+
+  test("insert overwrite to local dir from temp table") {
--- End diff --

In addition, also need to add negative test cases. For example, the path is 
illegal. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19082: [SPARK-21870][SQL] Split aggregation code into sm...

2017-08-31 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/19082#discussion_r136500779
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ---
@@ -244,6 +246,92 @@ case class HashAggregateExec(
 
   protected override val shouldStopRequired = false
 
+  // We assume a prefix has lower cases and a name has camel cases
+  private val variableName = "^[a-z]+_[a-zA-Z]+[0-9]*".r
+
+  // Returns true if a given name id belongs to this `CodegenContext`
+  private def isVariable(nameId: String): Boolean = nameId match {
+case variableName() => true
+case _ => false
+  }
+
+  // Extracts all the outer references for a given `aggExpr`. This result 
will be used to split
+  // aggregation into small functions.
+  private def getOuterReferences(
--- End diff --

`OuterReference` actually has special meaning in correlated subquery. This 
name can be confusing.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18931: [SPARK-21717][SQL] Decouple consume functions of physica...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18931
  
**[Test build #81307 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81307/testReport)**
 for PR 18931 at commit 
[`1101b2c`](https://github.com/apache/spark/commit/1101b2c085d78e6416401c78221e32fef018851b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136499419
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/InsertSuite.scala ---
@@ -534,4 +534,115 @@ class InsertIntoHiveTableSuite extends QueryTest with 
TestHiveSingleton with Bef
   }
 }
   }
+
+  test("insert overwrite to dir from hive metastore table") {
+withTempDir { dir =>
+  val path = dir.toURI.getPath
+
+  sql(s"INSERT OVERWRITE LOCAL DIRECTORY '${path}' SELECT * FROM src 
where key < 10")
+
+  sql(
+s"""
+   |INSERT OVERWRITE LOCAL DIRECTORY '${path}'
+   |STORED AS orc
+   |SELECT * FROM src where key < 10
+ """.stripMargin)
+
+  // use orc data source to check the data of path is right.
+  withTempView("orc_source") {
+sql(
+  s"""
+ |CREATE TEMPORARY VIEW orc_source
+ |USING org.apache.spark.sql.hive.orc
+ |OPTIONS (
+ |  PATH '${dir.getCanonicalPath}'
+ |)
+   """.stripMargin)
+
+checkAnswer(
+  sql("select * from orc_source"),
+  sql("select * from src where key < 10").collect())
+  }
+}
+  }
+
+  test("insert overwrite to local dir from temp table") {
--- End diff --

So far, both scenarios are still using the local file system, although we 
use `LOCAL`. Do your team has usage scenarios to verify the non-local mode 
works well?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18975: [SPARK-4131] Support "Writing data into the files...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/18975#discussion_r136497740
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ---
@@ -140,6 +141,10 @@ case class DataSourceAnalysis(conf: SQLConf) extends 
Rule[LogicalPlan] with Cast
 parts, query, overwrite, false) if parts.isEmpty =>
   InsertIntoDataSourceCommand(l, query, overwrite)
 
+case InsertIntoDir(_, storage, provider, query, overwrite)
+  if provider.isDefined && provider.get.toLowerCase(Locale.ROOT) != 
DDLUtils.HIVE_PROVIDER =>
+  InsertIntoDataSourceDirCommand(storage, provider.get, query, 
overwrite)
--- End diff --

We need to block both cases of `InsertIntoDir ` and 
`InsertIntoHiveDirCommand `: `cannot overwrite a path that is also being read 
from`. See the example:

https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala#L178-L187


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18647
  
Thank you @felixcheung and @holdenk.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18647
  
Merged to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18647: [SPARK-21789][PYTHON] Remove obsolete codes for p...

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/18647


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18647
  
I double checked these **`_split_schema_abstract`**, 
**`_parse_field_abstract`**, **`_parse_schema_abstract`** and 
**`_infer_schema_type`** are not used in a public API.

Under `./python/pyspark`:

**1. `_split_schema_abstract`**:

```
$ grep -r "_split_schema_abstract" .
```

shows

```
./sql/types.py:def _split_schema_abstract(s):
./sql/types.py:>>> _split_schema_abstract("a b  c")
./sql/types.py:>>> _split_schema_abstract("a(a b)")
./sql/types.py:>>> _split_schema_abstract("a b[] c{a b}")
./sql/types.py:>>> _split_schema_abstract(" ")
./sql/types.py:parts = _split_schema_abstract(s)
```

Non doctests / tests:

```
./sql/types.py:parts = _split_schema_abstract(s)
```

This is within **3. `_parse_schema_abstract`**:



https://github.com/apache/spark/blob/b56f79cc359d093d757af83171175cfd933162d1/python/pyspark/sql/types.py#L1274


**2. `_parse_field_abstract`**:

```
$ grep -r "_parse_field_abstract" .
```

shows

```
./sql/types.py:def _parse_field_abstract(s):
./sql/types.py:>>> _parse_field_abstract("a")
./sql/types.py:>>> _parse_field_abstract("b(c d)")
./sql/types.py:>>> _parse_field_abstract("a[]")
./sql/types.py:>>> _parse_field_abstract("a{[]}")
./sql/types.py:fields = [_parse_field_abstract(p) for p in parts]
```

Non doctests / tests:

```
fields = [_parse_field_abstract(p) for p in parts]
```

This is within **3. `_parse_schema_abstract`**:


https://github.com/apache/spark/blob/b56f79cc359d093d757af83171175cfd933162d1/python/pyspark/sql/types.py#L1275


**3. `_parse_schema_abstract`**:

```
$ grep -r "_parse_schema_abstract" .
```

shows

```
./sql/tests.py:from pyspark.sql.types import 
_parse_schema_abstract, _infer_schema_type
./sql/tests.py:schema = _parse_schema_abstract(abstract)
./sql/types.py:return StructField(name, 
_parse_schema_abstract(s[idx:]), True)
./sql/types.py:def _parse_schema_abstract(s):
./sql/types.py:>>> _parse_schema_abstract("a b  c")
./sql/types.py:>>> _parse_schema_abstract("a[b c] b{}")
./sql/types.py:>>> _parse_schema_abstract("c{} d{a b}")
./sql/types.py:>>> _parse_schema_abstract("a b(t)").fields[1]
./sql/types.py:return _parse_schema_abstract(s[1:-1])
./sql/types.py:return ArrayType(_parse_schema_abstract(s[1:-1]), 
True)
./sql/types.py:return MapType(NullType(), 
_parse_schema_abstract(s[1:-1]))
./sql/types.py:>>> schema = _parse_schema_abstract("a b c d")
./sql/types.py:>>> schema = _parse_schema_abstract("a[] b{c d}")
```

Non doctests / tests:

```
./sql/types.py:return StructField(name, 
_parse_schema_abstract(s[idx:]), True)
./sql/types.py:return _parse_schema_abstract(s[1:-1])
./sql/types.py:return ArrayType(_parse_schema_abstract(s[1:-1]), 
True)
./sql/types.py:return MapType(NullType(), 
_parse_schema_abstract(s[1:-1]))
```

These four are within **`2. _parse_field_abstract`** and within **`3. 
_parse_schema_abstract`**:


https://github.com/apache/spark/blob/b56f79cc359d093d757af83171175cfd933162d1/python/pyspark/sql/types.py#L1243


https://github.com/apache/spark/blob/b56f79cc359d093d757af83171175cfd933162d1/python/pyspark/sql/types.py#L1266


https://github.com/apache/spark/blob/b56f79cc359d093d757af83171175cfd933162d1/python/pyspark/sql/types.py#L1269


https://github.com/apache/spark/blob/b56f79cc359d093d757af83171175cfd933162d1/python/pyspark/sql/types.py#L1272


**4. `_infer_schema_type`**:

```
$ grep -r "_infer_schema_type"
```

shows

```
./sql/tests.py:from pyspark.sql.types import 
_parse_schema_abstract, _infer_schema_type
./sql/tests.py:typedSchema = _infer_schema_type(rdd.first(), schema)
./sql/types.py:def _infer_schema_type(obj, dataType):
./sql/types.py:>>> _infer_schema_type(row, schema)
./sql/types.py:>>> _infer_schema_type(row, schema)
./sql/types.py:eType = _infer_schema_type(obj[0], 
dataType.elementType)
./sql/types.py:return MapType(_infer_schema_type(k, 
dataType.keyType),
./sql/types.py:   _infer_schema_type(v, 
dataType.valueType))
./sql/types.py:fields = [StructField(f.name, _infer_schema_type(o, 
f.dataType), True)
```

Non doctests / tests:

```
./sql/types.py:eType = _infer_schema_type(obj[0], 
dataType.elementType)
./sql/types.py:return MapType(_infer_schema_type(k, 
dataType.keyTyp

[GitHub] spark pull request #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample AP...

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/18999


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18999
  
Thank you @viirya, @felixcheung, @rxin and @ueshin.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18999
  
Merged to master.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19090: [SPARK-21877][DEPLOY, WINDOWS] Handle quotes in Windows ...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/19090
  
Looks ok given the examples & syntax -  https://ss64.com/nt/cmd.html and 
https://technet.microsoft.com/en-us/library/cc771320(v=ws.11).aspx and my 
manual tests.

I think here is the very entry point to Windows users. So, will take a 
closer look few times more. Meanwhile, @minixalpha would you mind if I ask 
check if there are any potential corner cases as well?



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18869
  
**[Test build #81306 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81306/testReport)**
 for PR 18869 at commit 
[`b64c9e6`](https://github.com/apache/spark/commit/b64c9e67dfb34668ba93089d84f932778c005d7b).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/18869
  
retest this please.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18869: [SPARK-21654][SQL] Complement SQL predicates expression ...

2017-08-31 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/18869
  
@gatorsmile @HyukjinKwon #18818 is merged now. Can this PR go ahead?

As what #18818 did is to allow structs, arrays to be input expression for 
predicates, currently looks like we don't have explicit description to say 
those types are not supported.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19090: [SPARK-21877][DEPLOY, WINDOWS] Handle quotes in Windows ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19090
  
**[Test build #81305 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81305/testReport)**
 for PR 19090 at commit 
[`26fc756`](https://github.com/apache/spark/commit/26fc75670750b4710a2e0d97642ec1b52d51ffbb).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19090: [SPARK-21877][DEPLOY, WINDOWS] Handle quotes in Windows ...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/19090
  
ok to test


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #18931: [SPARK-21717][SQL] Decouple consume functions of ...

2017-08-31 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/18931#discussion_r136492055
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
 ---
@@ -149,14 +149,144 @@ trait CodegenSupport extends SparkPlan {
 
 ctx.freshNamePrefix = parent.variablePrefix
 val evaluated = evaluateRequiredVariables(output, inputVars, 
parent.usedInputs)
+
+// Under certain conditions, we can put the logic to consume the rows 
of this operator into
+// another function. So we can prevent a generated function too long 
to be optimized by JIT.
+// The conditions:
+// 1. The parent uses all variables in output. we can't defer variable 
evaluation when consume
+//in another function.
+// 2. The output variables are not empty. If it's empty, we don't 
bother to do that.
+// 3. We don't use row variable. The construction of row uses deferred 
variable evaluation. We
+//can't do it.
+val requireAllOutput = output.forall(parent.usedInputs.contains(_))
+val consumeFunc =
+  if (row == null && outputVars.nonEmpty && requireAllOutput) {
+constructDoConsumeFunction(ctx, inputVars)
+  } else {
+parent.doConsume(ctx, inputVars, rowVar)
+  }
 s"""
|${ctx.registerComment(s"CONSUME: ${parent.simpleString}")}
|$evaluated
-   |${parent.doConsume(ctx, inputVars, rowVar)}
+   |$consumeFunc
+ """.stripMargin
+  }
+
+  /**
+   * To prevent concatenated function growing too long to be optimized by 
JIT. Instead of inlining,
+   * we may put the consume logic of parent operator into a function and 
set this flag to `true`.
+   * The parent operator can know if its consume logic is inlined or in 
separated function.
+   */
+  private var doConsumeInFunc: Boolean = false
+
+  /**
+   * Returning true means we have at least one consume logic from child 
operator or this operator is
+   * separated in a function. If this is `true`, this operator shouldn't 
use `continue` statement to
+   * continue on next row, because its generated codes aren't enclosed in 
main while-loop.
+   *
+   * For example, we have generated codes for a query plan like:
+   *   Op1Exec
+   * Op2Exec
+   *   Op3Exec
+   *
+   * If we put the consume code of Op2Exec into a separated function, the 
generated codes are like:
+   *   while (...) {
+   * ... // logic of Op3Exec.
+   * Op2Exec_doConsume(...);
+   *   }
+   *   private boolean Op2Exec_doConsume(...) {
+   * ... // logic of Op2Exec to consume rows.
+   *   }
+   * For now, `doConsumeInChainOfFunc` of Op2Exec will be `true`.
+   *
+   * Notice for some operators like `HashAggregateExec`, it doesn't chain 
previous consume functions
+   * but begins with its produce framework. We should override 
`doConsumeInChainOfFunc` to return
+   * `false`.
+   */
+  protected def doConsumeInChainOfFunc: Boolean = {
+val codegenChildren = children.map(_.asInstanceOf[CodegenSupport])
+doConsumeInFunc || codegenChildren.exists(_.doConsumeInChainOfFunc)
+  }
+
+  /**
+   * The actual java statement this operator should use if there is a need 
to continue on next row
+   * in its `doConsume` codes.
+   *
+   *   while (...) {
+   * ...   // logic of Op3Exec.
+   * Op2Exec_doConsume(...);
+   *   }
+   *   private boolean Op2Exec_doConsume(...) {
+   * ...   // logic of Op2Exec to consume rows.
+   * continue; // Wrong. We can't use continue with the while-loop.
+   *   }
+   * In above code, we can't use `continue` in `Op2Exec_doConsume`.
+   *
+   * Instead, we do something like:
+   *   while (...) {
+   * ...  // logic of Op3Exec.
+   * boolean continueForLoop = Op2Exec_doConsume(...);
+   * if (continueForLoop) continue;
+   *   }
+   *   private boolean Op2Exec_doConsume(...) {
+   * ...  // logic of Op2Exec to consume rows.
+   * return true; // When we need to do continue, we return true.
+   *   }
+   */
+  protected def continueStatementInDoConsume: String = if 
(doConsumeInChainOfFunc) {
+"return true;";
+  } else {
+"continue;"
+  }
+
+  /**
+   * To prevent concatenated function growing too long to be optimized by 
JIT. We can separate the
+   * parent's `doConsume` codes of a `CodegenSupport` operator into a 
function to call.
+   */
+  protected def constructDoConsumeFunction(
+  ctx: CodegenContext,
+  inputVars: Seq[ExprCode]): String = {
+val (callingParams, arguList, inputVarsInFunc) =
--- End diff --
 

[GitHub] spark pull request #18931: [SPARK-21717][SQL] Decouple consume functions of ...

2017-08-31 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18931#discussion_r136491920
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/WholeStageCodegenExec.scala
 ---
@@ -149,14 +149,144 @@ trait CodegenSupport extends SparkPlan {
 
 ctx.freshNamePrefix = parent.variablePrefix
 val evaluated = evaluateRequiredVariables(output, inputVars, 
parent.usedInputs)
+
+// Under certain conditions, we can put the logic to consume the rows 
of this operator into
+// another function. So we can prevent a generated function too long 
to be optimized by JIT.
+// The conditions:
+// 1. The parent uses all variables in output. we can't defer variable 
evaluation when consume
+//in another function.
+// 2. The output variables are not empty. If it's empty, we don't 
bother to do that.
+// 3. We don't use row variable. The construction of row uses deferred 
variable evaluation. We
+//can't do it.
+val requireAllOutput = output.forall(parent.usedInputs.contains(_))
+val consumeFunc =
+  if (row == null && outputVars.nonEmpty && requireAllOutput) {
+constructDoConsumeFunction(ctx, inputVars)
+  } else {
+parent.doConsume(ctx, inputVars, rowVar)
+  }
 s"""
|${ctx.registerComment(s"CONSUME: ${parent.simpleString}")}
|$evaluated
-   |${parent.doConsume(ctx, inputVars, rowVar)}
+   |$consumeFunc
+ """.stripMargin
+  }
+
+  /**
+   * To prevent concatenated function growing too long to be optimized by 
JIT. Instead of inlining,
+   * we may put the consume logic of parent operator into a function and 
set this flag to `true`.
+   * The parent operator can know if its consume logic is inlined or in 
separated function.
+   */
+  private var doConsumeInFunc: Boolean = false
+
+  /**
+   * Returning true means we have at least one consume logic from child 
operator or this operator is
+   * separated in a function. If this is `true`, this operator shouldn't 
use `continue` statement to
+   * continue on next row, because its generated codes aren't enclosed in 
main while-loop.
+   *
+   * For example, we have generated codes for a query plan like:
+   *   Op1Exec
+   * Op2Exec
+   *   Op3Exec
+   *
+   * If we put the consume code of Op2Exec into a separated function, the 
generated codes are like:
+   *   while (...) {
+   * ... // logic of Op3Exec.
+   * Op2Exec_doConsume(...);
+   *   }
+   *   private boolean Op2Exec_doConsume(...) {
+   * ... // logic of Op2Exec to consume rows.
+   *   }
+   * For now, `doConsumeInChainOfFunc` of Op2Exec will be `true`.
+   *
+   * Notice for some operators like `HashAggregateExec`, it doesn't chain 
previous consume functions
+   * but begins with its produce framework. We should override 
`doConsumeInChainOfFunc` to return
+   * `false`.
+   */
+  protected def doConsumeInChainOfFunc: Boolean = {
+val codegenChildren = children.map(_.asInstanceOf[CodegenSupport])
+doConsumeInFunc || codegenChildren.exists(_.doConsumeInChainOfFunc)
+  }
+
+  /**
+   * The actual java statement this operator should use if there is a need 
to continue on next row
+   * in its `doConsume` codes.
+   *
+   *   while (...) {
+   * ...   // logic of Op3Exec.
+   * Op2Exec_doConsume(...);
+   *   }
+   *   private boolean Op2Exec_doConsume(...) {
+   * ...   // logic of Op2Exec to consume rows.
+   * continue; // Wrong. We can't use continue with the while-loop.
+   *   }
+   * In above code, we can't use `continue` in `Op2Exec_doConsume`.
+   *
+   * Instead, we do something like:
+   *   while (...) {
+   * ...  // logic of Op3Exec.
+   * boolean continueForLoop = Op2Exec_doConsume(...);
+   * if (continueForLoop) continue;
+   *   }
+   *   private boolean Op2Exec_doConsume(...) {
+   * ...  // logic of Op2Exec to consume rows.
+   * return true; // When we need to do continue, we return true.
+   *   }
+   */
+  protected def continueStatementInDoConsume: String = if 
(doConsumeInChainOfFunc) {
+"return true;";
+  } else {
+"continue;"
+  }
+
+  /**
+   * To prevent concatenated function growing too long to be optimized by 
JIT. We can separate the
+   * parent's `doConsume` codes of a `CodegenSupport` operator into a 
function to call.
+   */
+  protected def constructDoConsumeFunction(
+  ctx: CodegenContext,
+  inputVars: Seq[ExprCode]): String = {
+val (callingParams, arguList, inputVarsInFunc) =
--- End diff --
  

[GitHub] spark pull request #18966: [SPARK-21751][SQL] CodeGeneraor.splitExpressions ...

2017-08-31 Thread kiszk
Github user kiszk commented on a diff in the pull request:

https://github.com/apache/spark/pull/18966#discussion_r136491409
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -769,16 +769,27 @@ class CodegenContext {
   foldFunctions: Seq[String] => String = _.mkString("", ";\n", ";")): 
String = {
 val blocks = new ArrayBuffer[String]()
 val blockBuilder = new StringBuilder()
+val defaultMaxLines = 100
+val maxLines = if (SparkEnv.get != null) {
+  
SparkEnv.get.conf.getInt("spark.sql.codegen.expressions.maxCodegenLinesPerFunction",
--- End diff --

@gatorsmile Is there any progress?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19077: [SPARK-21860][core]Improve memory reuse for heap memory ...

2017-08-31 Thread 10110346
Github user 10110346 commented on the issue:

https://github.com/apache/spark/pull/19077
  
@jerryshao @JoshRosen yes, it would not generally be arbitrary sized 
allocations. Basically, we allocate memory in multiples of 4 or 8 bytes,even 
so, I think this change is also beneficial .
Also,I think this change  will not  impact on the code which leverage it, 
because `MemoryBlock` is not changed


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18999
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18999
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81303/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18999
  
**[Test build #81303 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81303/testReport)**
 for PR 18999 at commit 
[`f2608ab`](https://github.com/apache/spark/commit/f2608ab0ca1e64ce97d65bffb62a07935e4b3db8).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18704: [SPARK-20783][SQL] Create ColumnVector to abstract exist...

2017-08-31 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/18704
  
@cloud-fan  Resolved conflict, could you please review?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19082: [SPARK-21870][SQL] Split aggregation code into sm...

2017-08-31 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/19082#discussion_r136490017
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/HashAggregateExec.scala
 ---
@@ -244,6 +246,92 @@ case class HashAggregateExec(
 
   protected override val shouldStopRequired = false
 
+  // We assume a prefix has lower cases and a name has camel cases
+  private val variableName = "^[a-z]+_[a-zA-Z]+[0-9]*".r
+
+  // Returns true if a given name id belongs to this `CodegenContext`
+  private def isVariable(nameId: String): Boolean = nameId match {
+case variableName() => true
+case _ => false
+  }
+
+  // Extracts all the outer references for a given `aggExpr`. This result 
will be used to split
+  // aggregation into small functions.
+  private def getOuterReferences(
+  ctx: CodegenContext,
+  aggExpr: Expression,
+  subExprs: Map[Expression, SubExprEliminationState]): Set[(String, 
String)] = {
+val stack = mutable.Stack[Expression](aggExpr)
+val argSet = mutable.Set[(String, String)]()
+val addIfNotLiteral = (value: String, tpe: String) => {
+  if (isVariable(value)) {
+argSet += ((tpe, value))
+  }
+}
+while (stack.nonEmpty) {
+  stack.pop() match {
+case e if subExprs.contains(e) =>
+  val exprCode = subExprs(e)
+  addIfNotLiteral(exprCode.value, ctx.javaType(e.dataType))
+  addIfNotLiteral(exprCode.isNull, "boolean")
+  // Since the children possibly has common expressions, we push 
them here
+  stack.pushAll(e.children)
+case ref: BoundReference
+if ctx.currentVars != null && ctx.currentVars(ref.ordinal) != 
null =>
+  val argVal = ctx.currentVars(ref.ordinal).value
+  addIfNotLiteral(argVal, ctx.javaType(ref.dataType))
+  addIfNotLiteral(ctx.currentVars(ref.ordinal).isNull, "boolean")
+case _: BoundReference =>
+  argSet += (("InternalRow", ctx.INPUT_ROW))
+case e =>
+  stack.pushAll(e.children)
+  }
+}
+
+argSet.toSet
+  }
+
+  // Splits the aggregation into small functions because the HotSpot does 
not compile
+  // too long functions.
+  private def splitAggregateExpressions(
+  ctx: CodegenContext,
+  aggExprs: Seq[Expression],
+  evalAndUpdateCodes: Seq[String],
+  subExprs: Map[Expression, SubExprEliminationState],
+  otherArgs: Seq[(String, String)] = Seq.empty): Seq[String] = {
+aggExprs.zipWithIndex.map { case (aggExpr, i) =>
+  // The maximum number of parameters in Java methods is 255, so this 
method gives up splitting
+  // the code if the number goes over the limit.
+  // You can find more information about the limit in the JVM 
specification:
+  //   - The number of method parameters is limited to 255 by the 
definition of a method
+  // descriptor, where the limit includes one unit for this in the 
case of instance
+  // or interface method invocations.
+  val args = (getOuterReferences(ctx, aggExpr, subExprs) ++ 
otherArgs).toSeq
+
+  // This is for testing/benchmarking only
+  val maxParamNumInJavaMethod =
+  
sqlContext.getConf("spark.sql.codegen.aggregate.maxParamNumInJavaMethod", null) 
match {
--- End diff --

This is a test-only option, so I think we need not check that.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19077: [SPARK-21860][core]Improve memory reuse for heap ...

2017-08-31 Thread 10110346
Github user 10110346 commented on a diff in the pull request:

https://github.com/apache/spark/pull/19077#discussion_r136489896
  
--- Diff: 
common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
 ---
@@ -47,23 +47,29 @@ private boolean shouldPool(long size) {
 
   @Override
   public MemoryBlock allocate(long size) throws OutOfMemoryError {
-if (shouldPool(size)) {
+int arraySize = (int)((size + 7) / 8);
--- End diff --

But the type of input parameter for `roundNumberOfBytesToNearestWord` is 
`int`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19077: [SPARK-21860][core]Improve memory reuse for heap ...

2017-08-31 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/19077#discussion_r136487281
  
--- Diff: 
common/unsafe/src/main/java/org/apache/spark/unsafe/memory/HeapMemoryAllocator.java
 ---
@@ -47,23 +47,29 @@ private boolean shouldPool(long size) {
 
   @Override
   public MemoryBlock allocate(long size) throws OutOfMemoryError {
-if (shouldPool(size)) {
+int arraySize = (int)((size + 7) / 8);
--- End diff --

You might be able to use `ByteAraryMethods.roundNumberOfBytesToNearestWord` 
for this, which we'e done for similar rounding elsewhere. Makes it a bit easier 
to spot what's happening.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19077: [SPARK-21860][core]Improve memory reuse for heap memory ...

2017-08-31 Thread JoshRosen
Github user JoshRosen commented on the issue:

https://github.com/apache/spark/pull/19077
  
Just curious: do you know where are we allocating these close-in-size 
chunks of memory? I understand the motivation, but just curious to know what's 
causing this pattern. I think the original idea here was that most allocations 
would come from a small set of sizes (usually the page size, or a configurable 
buffer size) and would not generally be arbitrary sized allocations.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19100
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81300/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19100
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19100
  
**[Test build #81300 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81300/testReport)**
 for PR 19100 at commit 
[`7dbd810`](https://github.com/apache/spark/commit/7dbd81039a129276e75e51e4bf8cb7b7191e14e7).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19101: [SPARK-21884] [BACKPORT-2.2] [SPARK-21477] [SQL] Mark Lo...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19101
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19101: [SPARK-21884] [BACKPORT-2.2] [SPARK-21477] [SQL] Mark Lo...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19101
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81302/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19101: [SPARK-21884] [BACKPORT-2.2] [SPARK-21477] [SQL] Mark Lo...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19101
  
**[Test build #81302 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81302/testReport)**
 for PR 19101 at commit 
[`3ee18ce`](https://github.com/apache/spark/commit/3ee18cec3bba64149b437cb8d78dc6539b5aa7d7).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #17014: [SPARK-18608][ML] Fix double-caching in ML algorithms

2017-08-31 Thread zhengruifeng
Github user zhengruifeng commented on the issue:

https://github.com/apache/spark/pull/17014
  
@WeichenXu123 Sounds good. And since adding `handlePersistence` as a 
`ml.Param` may influences many algs (more than that in this PR), I think we may 
need more discussion  @MLnick @yanboliang 

And if we add `handlePersistence`, should we also add a param 
`intermediateStorageLevel` to let users choose the storagelevel (like 
(ALS)[https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/recommendation/ALS.scala#L216])?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #16774: [SPARK-19357][ML] Adding parallel model evaluatio...

2017-08-31 Thread WeichenXu123
Github user WeichenXu123 commented on a diff in the pull request:

https://github.com/apache/spark/pull/16774#discussion_r136482755
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tuning/CrossValidatorSuite.scala ---
@@ -120,6 +120,33 @@ class CrossValidatorSuite
 }
   }
 
+  test("cross validation with parallel evaluation") {
+val lr = new LogisticRegression
+val lrParamMaps = new ParamGridBuilder()
+  .addGrid(lr.regParam, Array(0.001, 1000.0))
+  .addGrid(lr.maxIter, Array(0, 3))
+  .build()
+val eval = new BinaryClassificationEvaluator
+val cv = new CrossValidator()
+  .setEstimator(lr)
+  .setEstimatorParamMaps(lrParamMaps)
+  .setEvaluator(eval)
+  .setNumFolds(2)
+  .setParallelism(1)
--- End diff --

OK I agree.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19060: [WIP][SQL] Add DataSourceSuite validating data so...

2017-08-31 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/19060#discussion_r136475026
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/sources/DataSourceSuite.scala ---
@@ -0,0 +1,151 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.sources
+
+import java.sql.{Date, Timestamp}
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.io.{IntWritable, NullWritable, Text}
+import org.apache.hadoop.mapred.JobConf
+import org.apache.hadoop.mapreduce._
+import org.apache.hadoop.mapreduce.lib.input.FileSplit
+import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
+import org.apache.orc.{OrcConf, TypeDescription}
+import org.apache.orc.mapred.OrcStruct
+import org.apache.orc.mapreduce.{OrcInputFormat, OrcOutputFormat}
+import org.apache.orc.storage.ql.io.sarg.{PredicateLeaf, 
SearchArgumentFactory}
+
+import org.apache.spark.sql.{Dataset, QueryTest, Row}
+import org.apache.spark.sql.hive.test.TestHiveSingleton
+import org.apache.spark.sql.test.SQLTestUtils
+
+/**
+ * Data Source qualification as Apache Spark Data Sources.
+ * - Apache Spark Data Type Value Limits
+ * - Predicate Push Down
+ */
+class DataSourceSuite
+  extends QueryTest
+  with SQLTestUtils
+  with TestHiveSingleton {
+
+  import testImplicits._
+
+  var df: Dataset[Row] = _
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+spark.conf.set("spark.sql.session.timeZone", "GMT")
+
+df = ((
+  false,
+  true,
+  Byte.MinValue,
+  Byte.MaxValue,
+  Short.MinValue,
+  Short.MaxValue,
+  Int.MinValue,
+  Int.MaxValue,
+  Long.MinValue,
+  Long.MaxValue,
+  Float.MinValue,
+  Float.MaxValue,
+  Double.MinValue,
+  Double.MaxValue,
+  Date.valueOf("0001-01-01"),
+  Date.valueOf("-12-31"),
+  new Timestamp(-6213576960L), // 0001-01-01 00:00:00.000
+  new Timestamp(25340230079L)  // -12-31 23:59:59.999
+) :: Nil).toDF()
+  }
+
+  override def afterAll(): Unit = {
+try {
+  spark.conf.unset("spark.sql.session.timeZone")
+} finally {
+  super.afterAll()
+}
+  }
+
+  Seq("parquet", "orc", "json", "csv").foreach { dataSource =>
--- End diff --

`json` and `csv` is considered.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #17014: [SPARK-18608][ML] Fix double-caching in ML algorithms

2017-08-31 Thread WeichenXu123
Github user WeichenXu123 commented on the issue:

https://github.com/apache/spark/pull/17014
  
@smurching Yes this should be added as a `ml.Param`, we should not add as 
an argument.
@zhengruifeng Would you mind update the PR according to our discussion 
result above ?
Make `handlePersistence` as a `ml.Param` (added to these algos, default 
value be `true`).
And we don't need to modify the `Predictor` and any other public interface 
for now.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #15334: [SPARK-10367][SQL] Support Parquet logical type INTERVAL

2017-08-31 Thread dilipbiswal
Github user dilipbiswal commented on the issue:

https://github.com/apache/spark/pull/15334
  
@gatorsmile Hi Sean, i tried apache-drill after looking through their 
documentation. And they are able to encode interval data into parquet.

```
0: jdbc:drill:zk=local> CREATE TABLE dfs.tmp.parquet_intervals AS 
. . . . . . . . . . . > (SELECT CAST( INTERVALYEAR_col as INTERVAL YEAR) 
INTERVALYEAR_col,
. . . . . . . . . . . > CAST( INTERVALDAY_col as INTERVAL DAY) 
INTERVALDAY_col,
. . . . . . . . . . . > CAST( INTERVAL_col as INTERVAL SECOND) INTERVAL_col 
. . . . . . . . . . . > FROM dfs.`/tmp/intervals.json`);
SLF4J: Failed to load class "org.slf4j.impl.StaticLoggerBinder".
SLF4J: Defaulting to no-operation (NOP) logger implementation
SLF4J: See http://www.slf4j.org/codes.html#StaticLoggerBinder for further 
details.
+---++
| Fragment  | Number of records written  |
+---++
| 0_0   | 3  |
+---++
```
Here is the schema of the written parquet file.
```
message root {
  optional fixed_len_byte_array(12) INTERVALYEAR_col (INTERVAL);
  optional fixed_len_byte_array(12) INTERVALDAY_col (INTERVAL);
  optional fixed_len_byte_array(12) INTERVAL_col (INTERVAL);
}

```
From presto's documentation, it seems like they also may be able to encode 
interval data. But i haven't tried. 

FYI - i also tried hive. Its not possible to encode interval data in 
parquet format through hive.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18647
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18647
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81304/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18647
  
**[Test build #81304 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81304/testReport)**
 for PR 18647 at commit 
[`83228cb`](https://github.com/apache/spark/commit/83228cbfd8e8e681bbbca2e970d55c917d4dbc89).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19099: [SPARK-21652][SQL] Fix rule confliction between I...

2017-08-31 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/19099#discussion_r136479931
  
--- Diff: sql/core/src/test/scala/org/apache/spark/sql/SQLQuerySuite.scala 
---
@@ -2663,4 +2664,31 @@ class SQLQuerySuite extends QueryTest with 
SharedSQLContext {
 // In unit test, Spark will fail the query if memory leak detected.
 spark.range(100).groupBy("id").count().limit(1).collect()
   }
+
+  test("SPARK-21652: rule confliction of InferFiltersFromConstraints and 
ConstantPropagation") {
+// Under test environment, throws Exception if the max iteration 
number is reached for an
--- End diff --

Just catching the exception is not enough for this test?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19099: [SPARK-21652][SQL] Fix rule confliction between I...

2017-08-31 Thread maropu
Github user maropu commented on a diff in the pull request:

https://github.com/apache/spark/pull/19099#discussion_r136479686
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -79,11 +79,11 @@ abstract class Optimizer(sessionCatalog: SessionCatalog)
   PushProjectionThroughUnion,
   ReorderJoin,
   EliminateOuterJoin,
+  InferFiltersFromConstraints,
--- End diff --

Why you changed this?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18647
  
**[Test build #81304 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81304/testReport)**
 for PR 18647 at commit 
[`83228cb`](https://github.com/apache/spark/commit/83228cbfd8e8e681bbbca2e970d55c917d4dbc89).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18999
  
**[Test build #81303 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81303/testReport)**
 for PR 18999 at commit 
[`f2608ab`](https://github.com/apache/spark/commit/f2608ab0ca1e64ce97d65bffb62a07935e4b3db8).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18999: [SPARK-21779][PYTHON] Simpler DataFrame.sample API in Py...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18999
  
retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18647: [SPARK-21789][PYTHON] Remove obsolete codes for parsing ...

2017-08-31 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/18647
  
retest this please


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19080: [SPARK-21865][SQL] simplify the distribution sema...

2017-08-31 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19080#discussion_r136477108
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 ---
@@ -284,24 +241,17 @@ case class RangePartitioning(ordering: 
Seq[SortOrder], numPartitions: Int)
   override def nullable: Boolean = false
   override def dataType: DataType = IntegerType
 
-  override def satisfies(required: Distribution): Boolean = required match 
{
-case UnspecifiedDistribution => true
-case OrderedDistribution(requiredOrdering) =>
-  val minSize = Seq(requiredOrdering.size, ordering.size).min
-  requiredOrdering.take(minSize) == ordering.take(minSize)
-case ClusteredDistribution(requiredClustering) =>
-  ordering.map(_.child).forall(x => 
requiredClustering.exists(_.semanticEquals(x)))
-case _ => false
-  }
-
-  override def compatibleWith(other: Partitioning): Boolean = other match {
-case o: RangePartitioning => this.semanticEquals(o)
-case _ => false
-  }
-
-  override def guarantees(other: Partitioning): Boolean = other match {
-case o: RangePartitioning => this.semanticEquals(o)
-case _ => false
+  override def satisfies(required: Distribution): Boolean = {
+super.satisfies(required) || {
+  required match {
+case OrderedDistribution(requiredOrdering) =>
+  val minSize = Seq(requiredOrdering.size, ordering.size).min
+  requiredOrdering.take(minSize) == ordering.take(minSize)
--- End diff --

BTW this doesn't cause any problems, because `OrderedDistribution` is only 
used for sort operator.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

2017-08-31 Thread janewangfb
Github user janewangfb commented on the issue:

https://github.com/apache/spark/pull/18975
  
Jenkin test please!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19080: [SPARK-21865][SQL] simplify the distribution sema...

2017-08-31 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/19080#discussion_r136476025
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/partitioning.scala
 ---
@@ -284,24 +241,17 @@ case class RangePartitioning(ordering: 
Seq[SortOrder], numPartitions: Int)
   override def nullable: Boolean = false
   override def dataType: DataType = IntegerType
 
-  override def satisfies(required: Distribution): Boolean = required match 
{
-case UnspecifiedDistribution => true
-case OrderedDistribution(requiredOrdering) =>
-  val minSize = Seq(requiredOrdering.size, ordering.size).min
-  requiredOrdering.take(minSize) == ordering.take(minSize)
-case ClusteredDistribution(requiredClustering) =>
-  ordering.map(_.child).forall(x => 
requiredClustering.exists(_.semanticEquals(x)))
-case _ => false
-  }
-
-  override def compatibleWith(other: Partitioning): Boolean = other match {
-case o: RangePartitioning => this.semanticEquals(o)
-case _ => false
-  }
-
-  override def guarantees(other: Partitioning): Boolean = other match {
-case o: RangePartitioning => this.semanticEquals(o)
-case _ => false
+  override def satisfies(required: Distribution): Boolean = {
+super.satisfies(required) || {
+  required match {
+case OrderedDistribution(requiredOrdering) =>
+  val minSize = Seq(requiredOrdering.size, ordering.size).min
+  requiredOrdering.take(minSize) == ordering.take(minSize)
--- End diff --

I noticed this too, the current logic only guarantees ordering but not 
clustering. But this is orthogonal to this PR and we can fix it in another PR.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19100
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81301/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19060: [WIP][SQL] Add DataSourceSuite validating data sources l...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19060
  
**[Test build #81298 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81298/testReport)**
 for PR 19060 at commit 
[`104f24c`](https://github.com/apache/spark/commit/104f24c9ad0743dc7c6329b4c0dde902e8e87de6).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19060: [WIP][SQL] Add DataSourceSuite validating data sources l...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19060
  
Merged build finished. Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19060: [WIP][SQL] Add DataSourceSuite validating data sources l...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19060
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81298/
Test PASSed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19100
  
**[Test build #81301 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81301/testReport)**
 for PR 19100 at commit 
[`7954c0b`](https://github.com/apache/spark/commit/7954c0b4f37334d7806c614a47868f27c18b1fe1).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19101: [SPARK-21884] [BACKPORT-2.2] [SPARK-21477] [SQL] Mark Lo...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19101
  
**[Test build #81302 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81302/testReport)**
 for PR 19101 at commit 
[`3ee18ce`](https://github.com/apache/spark/commit/3ee18cec3bba64149b437cb8d78dc6539b5aa7d7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19100
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19101: [SPARK-21884] [BACKPORT-2.2] [SPARK-21477] [SQL] ...

2017-08-31 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

https://github.com/apache/spark/pull/19101

[SPARK-21884] [BACKPORT-2.2] [SPARK-21477] [SQL] Mark LocalTableScanExec's 
input data transient

This PR is to backport https://github.com/apache/spark/pull/18686 for 
resolving the issue in https://github.com/apache/spark/pull/19094

---
 
## What changes were proposed in this pull request?
This PR is to mark the parameter `rows` and `unsafeRow` of 
LocalTableScanExec transient. It can avoid serializing the unneeded objects.

## How was this patch tested?
N/A

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gatorsmile/spark backport-21477

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/19101.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #19101


commit 3ee18cec3bba64149b437cb8d78dc6539b5aa7d7
Author: gatorsmile 
Date:   2017-08-31T23:45:01Z

fix.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18317: [SPARK-21113][CORE] Read ahead input stream to amortize ...

2017-08-31 Thread sitalkedia
Github user sitalkedia commented on the issue:

https://github.com/apache/spark/pull/18317
  
ping @zsxwing !


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18975
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/18975
  
**[Test build #81297 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81297/testReport)**
 for PR 18975 at commit 
[`e2db5e1`](https://github.com/apache/spark/commit/e2db5e1e0cc491480828328e07b7bb619dc05bbd).
 * This patch **fails SparkR unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18975: [SPARK-4131] Support "Writing data into the filesystem f...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/18975
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/81297/
Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19050: [SPARK-21835][SQL] RewritePredicateSubquery should not p...

2017-08-31 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/19050
  
ping @cloud-fan @hvanhovell Can you have time to review this? Thanks.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19100
  
**[Test build #81301 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81301/testReport)**
 for PR 19100 at commit 
[`7954c0b`](https://github.com/apache/spark/commit/7954c0b4f37334d7806c614a47868f27c18b1fe1).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL stat...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19100#discussion_r136473016
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala
 ---
@@ -117,4 +117,12 @@ class OptimizeMetadataOnlyQuerySuite extends QueryTest 
with SharedSQLContext {
 "select partcol1, max(partcol2) from srcpart where partcol1 = 0 group 
by rollup (partcol1)",
 "select partcol2 from (select partcol2 from srcpart where partcol1 = 0 
union all " +
   "select partcol2 from srcpart where partcol1 = 1) t group by 
partcol2")
+
+  test("SPARK-21884 Fix StackOverflowError on MetadataOnlyQuery") {
--- End diff --

Here, just add a test case from https://github.com/apache/spark/pull/19094 
for verifying the fix in https://github.com/apache/spark/pull/18686


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/19100
  
**[Test build #81300 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/81300/testReport)**
 for PR 19100 at commit 
[`7dbd810`](https://github.com/apache/spark/commit/7dbd81039a129276e75e51e4bf8cb7b7191e14e7).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL stat...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/19100#discussion_r136472066
  
--- Diff: sql/core/src/test/resources/sql-tests/results/cross-join.sql.out 
---
@@ -128,6 +128,7 @@ two 2   two 2   one 1   two 
2
 two2   two 2   three   3   two 2
 two2   two 2   two 2   two 2
 
+
--- End diff --

This was automatically added by`SQLQueryTestSuite`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: ...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/19100
  
@cloud-fan 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19100: [SPARK-21891] [SQL] Add TBLPROPERTIES to DDL stat...

2017-08-31 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

https://github.com/apache/spark/pull/19100

[SPARK-21891] [SQL] Add TBLPROPERTIES to DDL statement: CREATE TABLE USING

## What changes were proposed in this pull request?
Add `TBLPROPERTIES` to the DDL statement `CREATE TABLE USING`. 

After this change, the DDL becomes
```
CREATE [TEMPORARY] TABLE [IF NOT EXISTS] [db_name.]table_name
USING table_provider
[OPTIONS table_property_list]
[PARTITIONED BY (col_name, col_name, ...)]
[CLUSTERED BY (col_name, col_name, ...)
 [SORTED BY (col_name [ASC|DESC], ...)]
 INTO num_buckets BUCKETS
]
[LOCATION path]
[COMMENT table_comment]
[TBLPROPERTIES (property_name=property_value, ...)]
[[AS] select_statement];
```

## How was this patch tested?
Add a few tests

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/gatorsmile/spark 
addTablePropsToCreateTableUsing

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/spark/pull/19100.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #19100


commit 4be45f77f879b76817e85c1bb7307aea6772b332
Author: gatorsmile 
Date:   2017-08-31T23:03:47Z

fix.

commit 7dbd81039a129276e75e51e4bf8cb7b7191e14e7
Author: gatorsmile 
Date:   2017-08-31T23:30:13Z

fix.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19078: [SPARK-21862][ML] Add overflow check in PCA

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/19078


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19072: [SPARK-17139][ML][FOLLOW-UP] Add convenient metho...

2017-08-31 Thread asfgit
Github user asfgit closed the pull request at:

https://github.com/apache/spark/pull/19072


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19078: [SPARK-21862][ML] Add overflow check in PCA

2017-08-31 Thread jkbradley
Github user jkbradley commented on the issue:

https://github.com/apache/spark/pull/19078
  
LGTM
Merging with master
Thanks!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18686: [SPARK-21477] [SQL] [MINOR] Mark LocalTableScanExec's in...

2017-08-31 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/18686
  
Thank you so much!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19072: [SPARK-17139][ML][FOLLOW-UP] Add convenient metho...

2017-08-31 Thread jkbradley
Github user jkbradley commented on a diff in the pull request:

https://github.com/apache/spark/pull/19072#discussion_r136470877
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/classification/LogisticRegression.scala
 ---
@@ -1473,6 +1473,17 @@ sealed trait LogisticRegressionSummary extends 
Serializable {
   /** Returns weighted averaged f1-measure. */
   @Since("2.3.0")
   def weightedFMeasure: Double = multiclassMetrics.weightedFMeasure(1.0)
+
+  /**
+   * Convenient method for casting to binary logistic regression summary.
+   * This method will throws an Exception if the summary is not a binary 
summary.
--- End diff --

For future reference & being careful: This didn't correct "throws" -> 
"throw" but I'll just go ahead and merge this since it's a tiny nit.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18686: [SPARK-21477] [SQL] [MINOR] Mark LocalTableScanExec's in...

2017-08-31 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/18686
  
Sure. Will do it.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #18686: [SPARK-21477] [SQL] [MINOR] Mark LocalTableScanExec's in...

2017-08-31 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/18686
  
Yes. This is the fix.
@gatorsmile and @cloud-fan . Can we have this in branch-2.2, too?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19094: [SPARK-21884][SQL][BRANCH-2.2] Fix StackOverflowError on...

2017-08-31 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/19094
  
I close this issue. Thank you again.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark pull request #19094: [SPARK-21884][SQL][BRANCH-2.2] Fix StackOverflowE...

2017-08-31 Thread dongjoon-hyun
Github user dongjoon-hyun closed the pull request at:

https://github.com/apache/spark/pull/19094


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



[GitHub] spark issue #19099: [SPARK-21652][SQL] Fix rule confliction between InferFil...

2017-08-31 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/19099
  
Merged build finished. Test FAILed.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

-
To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org
For additional commands, e-mail: reviews-h...@spark.apache.org



  1   2   3   >