[GitHub] spark pull request: [SPARK-12997][SQL] Use cast expression to perf...

2016-02-21 Thread falaki
Github user falaki commented on a diff in the pull request:

https://github.com/apache/spark/pull/10980#discussion_r53594660
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVRelation.scala
 ---
@@ -226,11 +227,17 @@ object CSVRelation extends Logging {
   while (subIndex < safeRequiredIndices.length) {
 index = safeRequiredIndices(subIndex)
 val field = schemaFields(index)
-rowArray(subIndex) = CSVTypeCast.castTo(
-  indexSafeTokens(index),
-  field.dataType,
-  field.nullable,
-  params.nullValue)
+val value = indexSafeTokens(index)
+val dataType = field.dataType
+val isNull =
+  value == params.nullValue && field.nullable && 
!dataType.isInstanceOf[StringType]
+val safeValue = dataType match {
--- End diff --

Is this logic unit tested? I see you removed the unit test related to 
`DecimalType`. I suggest you move this to a utility function and unit test it. 
Would also be good to make sure we have one test in CSVSuite 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 pull request: [SPARK-13139][SQL] Create native DDL commands

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11048#discussion_r53594561
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala 
---
@@ -0,0 +1,420 @@
+/*
+ * 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.execution.commands
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, 
TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, 
SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.execution.commands._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types.StructType
+
+case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
+
+  def parsePartitionSpec(node: ASTNode): Option[Map[String, 
Option[String]]] = {
+node match {
+  case Token("TOK_PARTSPEC", partitions) =>
+val spec = partitions.map {
+  case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)),
+  Some(unquoteString(cleanIdentifier(constant.text
+  case Token("TOK_PARTVAL", ident :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)), None)
+}.toMap
+Some(spec)
+  case _ => None
+}
+  }
+
+  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node 
match {
+case Token("TOK_TABLEPROPERTIES", propsList) =>
+  propsList.flatMap {
+case Token("TOK_TABLEPROPLIST", props) =>
+  props.map {
+case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) 
:: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  (k, None)
+case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  val v = unquoteString(cleanIdentifier(value.text))
+  (k, Some(v))
+  }
+  }.toMap
+  }
+
+  override def isDefinedAt(node: ASTNode): Boolean = node.text == 
"TOK_ALTERTABLE"
+
+  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
+case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
+  val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
+  val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
+  val partition = partitionSpec.flatMap(parsePartitionSpec)
+  matchAlterTableCommands(v1, rest, tableIdent, partition)
+case _ =>
+  throw new NotImplementedError(v1.text)
+  }
+
+  def matchAlterTableCommands(
+  node: ASTNode,
+  nodes: Seq[ASTNode],
+  tableIdent: TableIdentifier,
+  partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes 
match {
+case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
+  val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
+  val renamedTableIdent: TableIdentifier = 
base.extractTableIdent(renamedTable)
+  AlterTableRename(tableIdent, renamedTableIdent)(node.source)
+
+case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
+  val setTableProperties = extractTableProps(args.head)
+  AlterTableSetProperties(
+tableIdent,
+setTableProperties)(node.source)
+
+case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
+  val dropTableProperties = extractTableProps(args.head)
+  val allowExisting = 

[GitHub] spark pull request: [SPARK-12617][PySpark]Move Py4jCallbackConnect...

2016-02-21 Thread sarathj
Github user sarathj commented on the pull request:

https://github.com/apache/spark/pull/10621#issuecomment-187058482
  
@zsxwing 
With 1.6 also I ran into the same exception -` java.io.IOException: 
py4j.Py4JException: Cannot obtain a new communication`. 

I would like to apply patch by using 0.9.1 version of py4j. Could you 
please let me know, how you applied the fix. Also I could not find 
`py4j-0.9.1-src.zip` link directly. Do I need to extract from the 
https://pypi.python.org/pypi/py4j and zip 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: [SPARK-13139][SQL] Create native DDL commands

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11048#discussion_r53593980
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala 
---
@@ -0,0 +1,420 @@
+/*
+ * 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.execution.commands
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, 
TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, 
SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.execution.commands._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types.StructType
+
+case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
+
+  def parsePartitionSpec(node: ASTNode): Option[Map[String, 
Option[String]]] = {
+node match {
+  case Token("TOK_PARTSPEC", partitions) =>
+val spec = partitions.map {
+  case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)),
+  Some(unquoteString(cleanIdentifier(constant.text
+  case Token("TOK_PARTVAL", ident :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)), None)
+}.toMap
+Some(spec)
+  case _ => None
+}
+  }
+
+  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node 
match {
+case Token("TOK_TABLEPROPERTIES", propsList) =>
+  propsList.flatMap {
+case Token("TOK_TABLEPROPLIST", props) =>
+  props.map {
+case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) 
:: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  (k, None)
+case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  val v = unquoteString(cleanIdentifier(value.text))
+  (k, Some(v))
+  }
+  }.toMap
+  }
+
+  override def isDefinedAt(node: ASTNode): Boolean = node.text == 
"TOK_ALTERTABLE"
+
+  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
+case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
+  val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
+  val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
+  val partition = partitionSpec.flatMap(parsePartitionSpec)
+  matchAlterTableCommands(v1, rest, tableIdent, partition)
+case _ =>
+  throw new NotImplementedError(v1.text)
+  }
+
+  def matchAlterTableCommands(
+  node: ASTNode,
+  nodes: Seq[ASTNode],
+  tableIdent: TableIdentifier,
+  partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes 
match {
+case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
+  val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
+  val renamedTableIdent: TableIdentifier = 
base.extractTableIdent(renamedTable)
+  AlterTableRename(tableIdent, renamedTableIdent)(node.source)
+
+case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
+  val setTableProperties = extractTableProps(args.head)
+  AlterTableSetProperties(
+tableIdent,
+setTableProperties)(node.source)
+
+case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
+  val dropTableProperties = extractTableProps(args.head)
+  val allowExisting = 

[GitHub] spark pull request: [SPARK-8000][SQL] Support for auto-detecting d...

2016-02-21 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11270#discussion_r53593587
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/ResolvedDataSource.scala
 ---
@@ -130,7 +131,28 @@ object ResolvedDataSource extends Logging {
   bucketSpec: Option[BucketSpec],
   provider: String,
   options: Map[String, String]): ResolvedDataSource = {
-val clazz: Class[_] = lookupDataSource(provider)
+val paths = {
+  val caseInsensitiveOptions = new CaseInsensitiveMap(options)
+  if (caseInsensitiveOptions.contains("paths") &&
+caseInsensitiveOptions.contains("path")) {
+throw new AnalysisException(s"Both path and paths options are 
present.")
+  }
+  caseInsensitiveOptions.get("paths")
+.map(_.split("(?

[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11048#discussion_r53593420
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala 
---
@@ -0,0 +1,420 @@
+/*
+ * 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.execution.commands
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, 
TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, 
SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.execution.commands._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types.StructType
+
+case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
+
+  def parsePartitionSpec(node: ASTNode): Option[Map[String, 
Option[String]]] = {
+node match {
+  case Token("TOK_PARTSPEC", partitions) =>
+val spec = partitions.map {
+  case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)),
+  Some(unquoteString(cleanIdentifier(constant.text
+  case Token("TOK_PARTVAL", ident :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)), None)
+}.toMap
+Some(spec)
+  case _ => None
+}
+  }
+
+  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node 
match {
+case Token("TOK_TABLEPROPERTIES", propsList) =>
+  propsList.flatMap {
+case Token("TOK_TABLEPROPLIST", props) =>
+  props.map {
+case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) 
:: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  (k, None)
+case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  val v = unquoteString(cleanIdentifier(value.text))
+  (k, Some(v))
+  }
+  }.toMap
+  }
+
+  override def isDefinedAt(node: ASTNode): Boolean = node.text == 
"TOK_ALTERTABLE"
+
+  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
+case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
+  val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
+  val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
+  val partition = partitionSpec.flatMap(parsePartitionSpec)
+  matchAlterTableCommands(v1, rest, tableIdent, partition)
+case _ =>
+  throw new NotImplementedError(v1.text)
+  }
+
+  def matchAlterTableCommands(
+  node: ASTNode,
+  nodes: Seq[ASTNode],
+  tableIdent: TableIdentifier,
+  partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes 
match {
+case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
+  val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
+  val renamedTableIdent: TableIdentifier = 
base.extractTableIdent(renamedTable)
+  AlterTableRename(tableIdent, renamedTableIdent)(node.source)
+
+case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
--- End diff --

Yeah, as above. I will update 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 

[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11048#discussion_r53593393
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala 
---
@@ -0,0 +1,420 @@
+/*
+ * 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.execution.commands
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, 
TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, 
SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.execution.commands._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types.StructType
+
+case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
+
+  def parsePartitionSpec(node: ASTNode): Option[Map[String, 
Option[String]]] = {
+node match {
+  case Token("TOK_PARTSPEC", partitions) =>
+val spec = partitions.map {
+  case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)),
+  Some(unquoteString(cleanIdentifier(constant.text
+  case Token("TOK_PARTVAL", ident :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)), None)
+}.toMap
+Some(spec)
+  case _ => None
+}
+  }
+
+  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node 
match {
+case Token("TOK_TABLEPROPERTIES", propsList) =>
+  propsList.flatMap {
+case Token("TOK_TABLEPROPLIST", props) =>
+  props.map {
+case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) 
:: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  (k, None)
+case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  val v = unquoteString(cleanIdentifier(value.text))
+  (k, Some(v))
+  }
+  }.toMap
+  }
+
+  override def isDefinedAt(node: ASTNode): Boolean = node.text == 
"TOK_ALTERTABLE"
+
+  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
+case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
+  val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
+  val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
+  val partition = partitionSpec.flatMap(parsePartitionSpec)
+  matchAlterTableCommands(v1, rest, tableIdent, partition)
+case _ =>
+  throw new NotImplementedError(v1.text)
+  }
+
+  def matchAlterTableCommands(
+  node: ASTNode,
+  nodes: Seq[ASTNode],
+  tableIdent: TableIdentifier,
+  partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes 
match {
+case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
--- End diff --

`rest` here is not alway Nil, it is possibly partition spec for alter table 
commands. This optional partition spec is after alter table command. Actually 
as we parse it before here, we don't need to use `rest` to catch it and can 
ignore it. I will update this.

For example,

TOK_ALTERTABLE 2, 1, 22, 12 
:- TOK_TABNAME 2, 5, 5, 12 
:  +- table_name 2, 5, 5, 12 
:- TOK_ALTERTABLE_COMPACT 3, 20, 22, 8 
:  +- 'MAJOR' 3, 22, 22, 8 
+- TOK_PARTSPEC 2, 7, 18, 34 
   :- TOK_PARTVAL 2, 10, 12, 34 
   :  :- dt 2, 10, 10, 34 
   :  +- '2008-08-08' 2, 12, 12, 37 
   

[GitHub] spark pull request: [MINOR][DOCS] Fix all typos in markdown files ...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11300#issuecomment-187055449
  
**[Test build #2565 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2565/consoleFull)**
 for PR 11300 at commit 
[`86aa742`](https://github.com/apache/spark/commit/86aa742c8f7aa2df09cdc7610b79be9b5bc258a2).


---
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: [SPARK-13139][SQL] Create native DDL commands

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11048#discussion_r53593184
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala 
---
@@ -0,0 +1,420 @@
+/*
+ * 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.execution.commands
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, 
TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, 
SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.execution.commands._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types.StructType
+
+case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
+
+  def parsePartitionSpec(node: ASTNode): Option[Map[String, 
Option[String]]] = {
+node match {
+  case Token("TOK_PARTSPEC", partitions) =>
+val spec = partitions.map {
+  case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)),
+  Some(unquoteString(cleanIdentifier(constant.text
+  case Token("TOK_PARTVAL", ident :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)), None)
+}.toMap
+Some(spec)
+  case _ => None
+}
+  }
+
+  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node 
match {
+case Token("TOK_TABLEPROPERTIES", propsList) =>
+  propsList.flatMap {
+case Token("TOK_TABLEPROPLIST", props) =>
+  props.map {
+case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) 
:: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  (k, None)
+case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  val v = unquoteString(cleanIdentifier(value.text))
+  (k, Some(v))
+  }
+  }.toMap
+  }
+
+  override def isDefinedAt(node: ASTNode): Boolean = node.text == 
"TOK_ALTERTABLE"
+
+  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
+case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
+  val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
+  val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
--- End diff --

Some alter table commands have partition spec defined. We should parse it 
at one place instead of doing this parsing at many places.


---
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: [SPARK-12757] Add block-level read/write locks...

2016-02-21 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/10705#discussion_r53592740
  
--- Diff: 
core/src/main/scala/org/apache/spark/storage/BlockInfoManager.scala ---
@@ -0,0 +1,356 @@
+/*
+ * 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.storage
+
+import java.lang
+import javax.annotation.concurrent.GuardedBy
+
+import scala.collection.JavaConverters._
+import scala.collection.mutable
+
+import com.google.common.cache.{CacheBuilder, CacheLoader, LoadingCache}
+import com.google.common.collect.ConcurrentHashMultiset
+
+import org.apache.spark.{Logging, TaskContext}
+
+
+/**
+ * Tracks metadata for an individual block.
+ *
+ * @param level the block's storage level. This is the requested 
persistence level, not the
+ *  effective storage level of the block (i.e. if this is 
MEMORY_AND_DISK, then this
+ *  does not imply that the block is actually resident in 
memory).
+ * @param tellMaster whether state changes for this block should be 
reported to the master. This
+ *   is true for most blocks, but is false for broadcast 
blocks.
+ */
+private[storage] class BlockInfo(val level: StorageLevel, val tellMaster: 
Boolean) {
+
+  /**
+   * The size of the block (in bytes)
+   */
+  var size: Long = 0
+
+  /**
+   * The number of times that this block has been locked for reading.
+   */
+  var readerCount: Int = 0
+
+  /**
+   * The task attempt id of the task which currently holds the write lock 
for this block, or -1
+   * if this block is not locked for writing.
+   */
+  var writerTask: Long = -1
+
+  // Invariants:
+  // (writerTask != -1) implies (readerCount == 0)
+  // (readerCount != 0) implies (writerTask == -1)
+  // TODO: add assertions around every method
+
+  /**
+   * True if this block has been removed from the BlockManager and false 
otherwise.
+   * This field is used to communicate block deletion to blocked readers / 
writers (see its usage
+   * in [[BlockInfoManager]]).
+   */
+  var removed: Boolean = false
+
+  // TODO: Add timestamps on lock acquisitions
+}
+// In debugging mode, check that locks haven't been held for too long.
+// Every few minutes, dump debug info.
+
+/**
+ * Component of the [[BlockManager]] which tracks metadata for blocks and 
manages block locking.
+ *
+ * The locking interface exposed by this class is readers-writers lock. 
Every lock acquisition is
+ * automatically associated with a running task and locks are 
automatically released upon task
+ * completion or failure.
+ *
+ * This class is thread-safe.
+ */
+private[storage] class BlockInfoManager extends Logging {
+
+  private type TaskAttemptId = Long
+
+  /**
+   * Used to look up metadata for individual blocks. Entries are added to 
this map via an atomic
+   * set-if-not-exists operation ([[lockNewBlockForWriting()]]) and are 
removed
+   * by [[removeBlock()]].
+   */
+  @GuardedBy("this")
+  private[this] val infos = new mutable.HashMap[BlockId, BlockInfo]
+
+  /**
+   * Tracks the set of blocks that each task has locked for writing.
+   */
+  @GuardedBy("this")
+  private[this] val writeLocksByTask =
+new mutable.HashMap[TaskAttemptId, mutable.Set[BlockId]]
+  with mutable.MultiMap[TaskAttemptId, BlockId]
+
+  /**
+   * Tracks the set of blocks that each task has locked for reading, along 
with the number of times
+   * that a block has been locked (since our read locks are re-entrant). 
This is thread-safe.
+   */
+  private[this] val readLocksByTask: LoadingCache[lang.Long, 
ConcurrentHashMultiset[BlockId]] = {
+// We need to explicitly box as java.lang.Long to avoid a type 
mismatch error:
+val loader = new CacheLoader[java.lang.Long, 
ConcurrentHashMultiset[BlockId]] {

[GitHub] spark pull request: [SPARK-13139][SQL] Create native DDL commands

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11048#discussion_r53592672
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/commands/parsers.scala 
---
@@ -0,0 +1,420 @@
+/*
+ * 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.execution.commands
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.sql.{AnalysisException, SaveMode}
+import org.apache.spark.sql.catalyst.{CatalystQl, PlanParser, 
TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
+import org.apache.spark.sql.catalyst.expressions.{Ascending, Descending}
+import org.apache.spark.sql.catalyst.parser.{ASTNode, ParserConf, 
SimpleParserConf}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, 
OneRowRelation}
+import org.apache.spark.sql.catalyst.plans.logical
+import org.apache.spark.sql.execution.commands._
+import org.apache.spark.sql.execution.datasources._
+import org.apache.spark.sql.types.StructType
+
+case class AlterTableCommandParser(base: CatalystQl) extends PlanParser {
+
+  def parsePartitionSpec(node: ASTNode): Option[Map[String, 
Option[String]]] = {
+node match {
+  case Token("TOK_PARTSPEC", partitions) =>
+val spec = partitions.map {
+  case Token("TOK_PARTVAL", ident :: constant :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)),
+  Some(unquoteString(cleanIdentifier(constant.text
+  case Token("TOK_PARTVAL", ident :: Nil) =>
+(unquoteString(cleanIdentifier(ident.text)), None)
+}.toMap
+Some(spec)
+  case _ => None
+}
+  }
+
+  def extractTableProps(node: ASTNode): Map[String, Option[String]] = node 
match {
+case Token("TOK_TABLEPROPERTIES", propsList) =>
+  propsList.flatMap {
+case Token("TOK_TABLEPROPLIST", props) =>
+  props.map {
+case Token("TOK_TABLEPROPERTY", key :: Token("TOK_NULL", Nil) 
:: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  (k, None)
+case Token("TOK_TABLEPROPERTY", key :: value :: Nil) =>
+  val k = unquoteString(cleanIdentifier(key.text))
+  val v = unquoteString(cleanIdentifier(value.text))
+  (k, Some(v))
+  }
+  }.toMap
+  }
+
+  override def isDefinedAt(node: ASTNode): Boolean = node.text == 
"TOK_ALTERTABLE"
+
+  override def apply(v1: ASTNode): LogicalPlan = v1.children match {
+case (tabName @ Token("TOK_TABNAME", _)) :: rest =>
+  val tableIdent: TableIdentifier = base.extractTableIdent(tabName)
+  val partitionSpec = base.getClauseOption("TOK_PARTSPEC", v1.children)
+  val partition = partitionSpec.flatMap(parsePartitionSpec)
+  matchAlterTableCommands(v1, rest, tableIdent, partition)
+case _ =>
+  throw new NotImplementedError(v1.text)
+  }
+
+  def matchAlterTableCommands(
+  node: ASTNode,
+  nodes: Seq[ASTNode],
+  tableIdent: TableIdentifier,
+  partition: Option[Map[String, Option[String]]]): LogicalPlan = nodes 
match {
+case rename @ Token("TOK_ALTERTABLE_RENAME", renameArgs) :: rest =>
+  val renamedTable = base.getClause("TOK_TABNAME", renameArgs)
+  val renamedTableIdent: TableIdentifier = 
base.extractTableIdent(renamedTable)
+  AlterTableRename(tableIdent, renamedTableIdent)(node.source)
+
+case Token("TOK_ALTERTABLE_PROPERTIES", args) :: rest =>
+  val setTableProperties = extractTableProps(args.head)
+  AlterTableSetProperties(
+tableIdent,
+setTableProperties)(node.source)
+
+case Token("TOK_ALTERTABLE_DROPPROPERTIES", args) :: rest =>
+  val dropTableProperties = extractTableProps(args.head)
+  val allowExisting = 

[GitHub] spark pull request: [SPARK-12811] [ML] Estimator for Generalized L...

2016-02-21 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/11136#discussion_r53592447
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/regression/GeneralizedLinearRegression.scala
 ---
@@ -0,0 +1,547 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.regression
+
+import breeze.stats.distributions.{Gaussian => GD}
+
+import org.apache.spark.{Logging, SparkException}
+import org.apache.spark.annotation.{Experimental, Since}
+import org.apache.spark.ml.PredictorParams
+import org.apache.spark.ml.feature.Instance
+import org.apache.spark.ml.optim._
+import org.apache.spark.ml.param._
+import org.apache.spark.ml.param.shared._
+import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.mllib.linalg.{BLAS, Vector}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{DataFrame, Row}
+import org.apache.spark.sql.functions._
+
+/**
+ * Params for Generalized Linear Regression.
+ */
+private[regression] trait GeneralizedLinearRegressionParams extends 
PredictorParams
+  with HasFitIntercept with HasMaxIter with HasTol with HasRegParam with 
HasWeightCol
+  with HasSolver with Logging {
+
+  /**
+   * Param for the name of family which is a description of the error 
distribution
+   * to be used in the model.
+   * Supported options: "gaussian", "binomial", "poisson" and "gamma".
+   * @group param
+   */
+  @Since("2.0.0")
+  final val family: Param[String] = new Param(this, "family",
+"the name of family which is a description of the error distribution 
to be used in the model",
+
ParamValidators.inArray[String](GeneralizedLinearRegression.supportedFamilies.toArray))
+
+  /** @group getParam */
+  @Since("2.0.0")
+  def getFamily: String = $(family)
+
+  /**
+   * Param for the name of the model link function.
+   * Supported options: "identity", "log", "inverse", "logit", "probit", 
"cloglog" and "sqrt".
+   * @group param
+   */
+  @Since("2.0.0")
+  final val link: Param[String] = new Param(this, "link", "the name of the 
model link function",
+
ParamValidators.inArray[String](GeneralizedLinearRegression.supportedLinks.toArray))
+
+  /** @group getParam */
+  @Since("2.0.0")
+  def getLink: String = $(link)
+
+  @Since("2.0.0")
+  override def validateParams(): Unit = {
+if (isDefined(link)) {
+  
require(GeneralizedLinearRegression.supportedFamilyLinkPairs.contains($(family) 
-> $(link)),
+s"Generalized Linear Regression with ${$(family)} family does not 
support ${$(link)} " +
+  s"link function.")
+}
+  }
+}
+
+/**
+ * :: Experimental ::
+ *
+ * Fit a Generalized Linear Model 
([[https://en.wikipedia.org/wiki/Generalized_linear_model]])
+ * specified by giving a symbolic description of the linear predictor and
+ * a description of the error distribution.
+ */
+@Experimental
+@Since("2.0.0")
+class GeneralizedLinearRegression @Since("2.0.0") (@Since("2.0.0") 
override val uid: String)
+  extends Regressor[Vector, GeneralizedLinearRegression, 
GeneralizedLinearRegressionModel]
+  with GeneralizedLinearRegressionParams with Logging {
+
+  @Since("2.0.0")
+  def this() = this(Identifiable.randomUID("genLinReg"))
+
+  /**
+   * Set the name of family which is a description of the error 
distribution
+   * to be used in the model.
+   * @group setParam
+   */
+  @Since("2.0.0")
+  def setFamily(value: String): this.type = set(family, value)
+
+  /**
+   * Set the name of the model link function.
+   * @group setParam
+   */
+  @Since("2.0.0")
+  def setLink(value: String): this.type = set(link, value)
+
+  /**
+   * Set if we should fit the intercept.
+   * Default is true.
+   * @group setParam
+   */
+  @Since("2.0.0")
+  def setFitIntercept(value: 

[GitHub] spark pull request: [SPARK-13033][ML][PySpark] Add import/export f...

2016-02-21 Thread Wenpei
Github user Wenpei commented on the pull request:

https://github.com/apache/spark/pull/11000#issuecomment-187047801
  
@yanboliang I complete this pr, please take a look


---
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: [MINOR][DOCS] Fix all typos in markdown files ...

2016-02-21 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the pull request:

https://github.com/apache/spark/pull/11300#issuecomment-187046857
  
Hi, @rxin . 
I finished reviewing all files under the `docs` module and related stuff. 
I think it's all typos for now. 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: [SPARK-12720] [SQL] SQL Generation Support for...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11283#issuecomment-187045768
  
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: [SPARK-13236] SQL Generation for Set Operation...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11195#issuecomment-187045685
  
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: [SPARK-13263] [SQL] SQL Generation Support for...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11148#issuecomment-187045605
  
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: [MINOR][DOCS] Fix all typos in markdown files ...

2016-02-21 Thread dongjoon-hyun
GitHub user dongjoon-hyun opened a pull request:

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

[MINOR][DOCS] Fix all typos in markdown files of `doc` and similar patterns 
in other comments

## What changes were proposed in this pull request?

This PR tries to fix all typos in all markdown files under `docs` module, 
and fixes similar typos in other comments, too.

## How was the this patch tested?

manual tests.

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

$ git pull https://github.com/dongjoon-hyun/spark minor_fix_typos

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

https://github.com/apache/spark/pull/11300.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 #11300


commit 86aa742c8f7aa2df09cdc7610b79be9b5bc258a2
Author: Dongjoon Hyun 
Date:   2016-02-22T07:00:52Z

[MINOR][DOCS] Fix all typos in markdown files of `doc` and similar patterns 
in other comments.




---
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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187040244
  
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: [SPARK-8000][SQL] Support for auto-detecting d...

2016-02-21 Thread HyukjinKwon
Github user HyukjinKwon commented on the pull request:

https://github.com/apache/spark/pull/11270#issuecomment-187032665
  
I submitted some more commits. In summary, 

1. Added a `DataSourceDetect` class separatly.
2. Now, it only picks a single file. If the given path is directory, it 
goes deep and picks a single file if the directory does not have the extension.
3. I did not remove `sqlContext.conf.defaultDataSourceName` here as I see 
it is referred from relatively a lot of classes (so I thought I could do this 
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 pull request: [SPARK-8000][SQL] Support for auto-detecting d...

2016-02-21 Thread HyukjinKwon
Github user HyukjinKwon commented on the pull request:

https://github.com/apache/spark/pull/11270#issuecomment-187031347
  
I submitted some more commits. In summary, 

1. Added a `DataSourceDetect` class separatly.
2. Now, it only picks a single file. If the given path is directory, it 
goes deep and picks a single file if the directory does not have the extension.
3. I did not remove `sqlContext.conf.defaultDataSourceName` here as I see 
it is referred from relatively a lot of classes (so I thought I could do this 
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 pull request: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-187031165
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51658/
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 pull request: [SPARK-8000][SQL] Support for auto-detecting d...

2016-02-21 Thread HyukjinKwon
Github user HyukjinKwon commented on the pull request:

https://github.com/apache/spark/pull/11270#issuecomment-187031213
  
I submitted some more commits. In summary, 

1. Added a `DataSourceDetect` class separatly.
2. Now, it only picks a single file. If the given path is directory, it 
goes deep and picks a single file if the directory does not have the extension.
3. I did not remove `sqlContext.conf.defaultDataSourceName` here as I see 
it is referred from relatively a lot of classes (so I thought I could do this 
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 pull request: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-187031164
  
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 pull request: [SPARK-12757] Add block-level read/write locks...

2016-02-21 Thread zsxwing
Github user zsxwing commented on a diff in the pull request:

https://github.com/apache/spark/pull/10705#discussion_r53589538
  
--- Diff: 
core/src/main/scala/org/apache/spark/broadcast/TorrentBroadcast.scala ---
@@ -127,16 +127,23 @@ private[spark] class TorrentBroadcast[T: 
ClassTag](obj: T, id: Long)
   def getRemote: Option[ByteBuffer] = bm.getRemoteBytes(pieceId).map { 
block =>
 // If we found the block from remote executors/driver's 
BlockManager, put the block
 // in this executor's BlockManager.
-SparkEnv.get.blockManager.putBytes(
-  pieceId,
-  block,
-  StorageLevel.MEMORY_AND_DISK_SER,
-  tellMaster = true)
+bm.putBytes(pieceId, block, StorageLevel.MEMORY_AND_DISK_SER, 
tellMaster = true)
 block
   }
   val block: ByteBuffer = getLocal.orElse(getRemote).getOrElse(
 throw new SparkException(s"Failed to get $pieceId of 
$broadcastId"))
   blocks(pid) = block
+  Option(TaskContext.get()) match {
--- End diff --

Could you add a util method for these codes? I saw they appear multiple 
times.


---
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: [SPARK-13429] [MLlib] Unify Logistic Regressio...

2016-02-21 Thread yanboliang
GitHub user yanboliang opened a pull request:

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

[SPARK-13429] [MLlib] Unify Logistic Regression convergence tolerance of ML 
& MLlib

## What changes were proposed in this pull request?
In order to provide better and consistent result, let's change the default 
value of MLlib ```LogisticRegressionWithLBFGS convergenceTol``` from 1E-4 to 
1E-6 which will be equal to ML ```LogisticRegression```.
cc @dbtsai 
## How was the this patch tested?
unit tests



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

$ git pull https://github.com/yanboliang/spark spark-13429

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

https://github.com/apache/spark/pull/11299.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 #11299


commit f6b7b8bf38a8bc4fcc18f5126e9b1fdf2211cb72
Author: Yanbo Liang 
Date:   2016-02-22T05:54:47Z

Unify Logistic Regression convergence tolerance of ML & MLlib




---
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-187030953
  
**[Test build #51658 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51658/consoleFull)**
 for PR 11298 at commit 
[`50071e2`](https://github.com/apache/spark/commit/50071e2cf9c76fd1919c437500b82bfd287b5f95).
 * 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: [SPARK-8000][SQL] Support for auto-detecting d...

2016-02-21 Thread HyukjinKwon
Github user HyukjinKwon commented on the pull request:

https://github.com/apache/spark/pull/11270#issuecomment-187030511
  
I submitted some more commits. In summary, 

1. Added a `DataSourceDetect` class separatly.
2. Now, it only picks a single file. If the given path is directory, it 
goes deep and picks a single file if the directory does not have the extension.
3. I did not remove `sqlContext.conf.defaultDataSourceName` here as I see 
it is referred from relatively a lot of classes (so I thought I could do this 
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 pull request: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187028985
  
**[Test build #2564 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2564/consoleFull)**
 for PR 11296 at commit 
[`2614636`](https://github.com/apache/spark/commit/26146364a86b8c74cb9f0805a728df2581aefc60).


---
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187026346
  
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187026349
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51660/
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: [SPARK-12988][SQL] Can't drop columns that con...

2016-02-21 Thread dilipbiswal
Github user dilipbiswal commented on the pull request:

https://github.com/apache/spark/pull/10943#issuecomment-187018074
  
@cloud-fan Hi Wenchen, can you please advice on what is the next step for 
this PR ? I am thinking that it may require more discussion to decide if we 
need top keep or remove the df.drop(Column) interface.
What do you think ?


---
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: [SPARK-13379] [MLlib] Fix MLlib LogisticRegres...

2016-02-21 Thread yanboliang
Github user yanboliang commented on the pull request:

https://github.com/apache/spark/pull/11258#issuecomment-187017636
  
@dbtsai Thanks! I will send another PR for consistent convergence tolerance.


---
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: [SPARK-13427][SQL] Support USING clause in JOI...

2016-02-21 Thread dilipbiswal
Github user dilipbiswal commented on the pull request:

https://github.com/apache/spark/pull/11297#issuecomment-187014299
  
@rxin @adrian-wang Can you please review the implementation and let me know 
your comments.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 pull request: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187011421
  
**[Test build #51659 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51659/consoleFull)**
 for PR 11260 at commit 
[`03dac45`](https://github.com/apache/spark/commit/03dac455b1a7e3ef586d54d210bca0f762089624).
 * This patch **fails Scala style 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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187011434
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51659/
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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187011428
  
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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187010823
  
**[Test build #51659 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51659/consoleFull)**
 for PR 11260 at commit 
[`03dac45`](https://github.com/apache/spark/commit/03dac455b1a7e3ef586d54d210bca0f762089624).


---
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-187009069
  
cc @davies @cloud-fan . This PR is like an incremental patch with few lines 
of code changes. I am not sure if the whole rewrite is worthy for resolving the 
above two issues only when there exist the other operators between `Sort` and 
`Aggregate`? After all, users can manually correct their query if they hit a 
resolution issue when missing attributes in Sort happen. We already cover most 
of cases. : ) 

Let me know if you want a whole rewrite or just an incremental fix. 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 pull request: [SPARK-13080] [SQL] Implement new Catalog API ...

2016-02-21 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11293#discussion_r53585835
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveCatalog.scala ---
@@ -0,0 +1,293 @@
+/*
+ * 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
+
+import scala.util.control.NonFatal
+
+import org.apache.hadoop.hive.ql.metadata.HiveException
+import org.apache.thrift.TException
+
+import org.apache.spark.Logging
+import org.apache.spark.sql.AnalysisException
+import org.apache.spark.sql.catalyst.analysis.NoSuchItemException
+import org.apache.spark.sql.catalyst.catalog._
+import org.apache.spark.sql.hive.client.HiveClient
+
+
+/**
+ * A persistent implementation of the system catalog using Hive.
+ * All public methods must be synchronized for thread-safety.
+ */
+private[spark] class HiveCatalog(client: HiveClient) extends Catalog with 
Logging {
+  import Catalog._
+
+  // Exceptions thrown by the hive client that we would like to wrap
+  private val clientExceptions = Set(
+classOf[HiveException].getCanonicalName,
+classOf[TException].getCanonicalName)
+
+  /**
+   * Whether this is an exception thrown by the hive client that should be 
wrapped.
+   *
+   * Due to classloader isolation issues, pattern matching won't work here 
so we need
+   * to compare the canonical names of the exceptions, which we assume to 
be stable.
+   */
+  private def isClientException(e: Throwable): Boolean = {
+var temp: Class[_] = e.getClass
+var found = false
+while (temp != null && !found) {
+  found = clientExceptions.contains(temp.getCanonicalName)
+  temp = temp.getSuperclass
+}
+found
+  }
+
+  /**
+   * Run some code involving `client` in a [[synchronized]] block and wrap 
certain
+   * exceptions thrown in the process in [[AnalysisException]].
+   */
+  private def withClient[T](body: => T): T = synchronized {
--- End diff --

cc @andrewor14 why does this one need to be synchronized? 


---
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/11298#discussion_r53585798
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ---
@@ -607,14 +607,21 @@ class Analyzer(
 def apply(plan: LogicalPlan): LogicalPlan = plan resolveOperators {
   // Skip sort with aggregate. This will be handled in 
ResolveAggregateFunctions
   case sa @ Sort(_, _, child: Aggregate) => sa
--- End diff --

The above two issues have been covered in the rule 
`ResolveAggregateFunctions` when `Aggregate` is the child of `Sort`. Thus, the 
current PR keeps the existing logics without any change. 

This PR is to introduce a fix for aggregate functions when there exists the 
other operators between `Aggregate` and `Sort`.


---
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-187005048
  
# Issue 2: mixture of aliases and real columns in order by clause
```SQL
SELECT key as k, value as v, sum(value) FROM src GROUP BY key, value ORDER 
BY k, value
```
```
== Parsed Logical Plan ==
'Sort ['k ASC,'value ASC], true
+- 'Aggregate ['key,'value], [unresolvedalias('key AS 
k#47,None),unresolvedalias('value AS 
v#48,None),unresolvedalias('sum('value),None)]
   +- 'UnresolvedRelation `src`, None
```
In the above case, we need to replace the actual column name `value` by the 
alias name `v`. Otherwise, we have to introduce `k` in all the nodes between 
`Sort` and `Aggregate`.
```
Project [k#47,v#48,_c2#54L]
+- Sort [k#47 ASC,v#48 ASC], true
   +- Aggregate [key#45,value#46], [key#45 AS k#47,value#46 AS 
v#48,(sum(cast(value#46 as bigint)),mode=Complete,isDistinct=false) AS _c2#54L]
  +- Subquery src
 +- Project [_1#43 AS key#45,_2#44 AS value#46]
+- LocalRelation [_1#43,_2#44], [[1,1],[-1,1]]
```


---
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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187005053
  
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: [SPARK-13233][SQL][WIP] Python Dataset

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7#issuecomment-187005041
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51652/
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: [SPARK-13233][SQL][WIP] Python Dataset

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7#issuecomment-187005038
  
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: [SPARK-13233][SQL][WIP] Python Dataset

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7#issuecomment-187004772
  
**[Test build #51652 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51652/consoleFull)**
 for PR 7 at commit 
[`8c32d31`](https://github.com/apache/spark/commit/8c32d311b7189b9a7a227a89d84681e97d2997a3).
 * This patch **fails PySpark 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 pull request: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187002198
  
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187002350
  
You need to update mima.



---
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187002203
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51655/
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-187002073
  
**[Test build #51655 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51655/consoleFull)**
 for PR 11296 at commit 
[`fb4d68a`](https://github.com/apache/spark/commit/fb4d68a1676dd31d79dbd5c7c074c1d9a09e103a).
 * This patch **fails MiMa 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: [SPARK-13389] [SparkR] SparkR support first/la...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11267#issuecomment-187001699
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51656/
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 pull request: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187001554
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51657/
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: [SPARK-13389] [SparkR] SparkR support first/la...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11267#issuecomment-187001694
  
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 pull request: [SPARK-13389] [SparkR] SparkR support first/la...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11267#issuecomment-187001455
  
**[Test build #51656 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51656/consoleFull)**
 for PR 11267 at commit 
[`6e78f7c`](https://github.com/apache/spark/commit/6e78f7c4a6145a5be57b1133555b8a4172d90129).
 * 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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11260#issuecomment-187001550
  
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-187001189
  
# Issue 1:
```SQL
SELECT MAX(value) FROM src GROUP BY key + 1 ORDER BY key + 1
```
`key + 1` is not an aggregated function, but we still need to push it down 
into `Aggregate`, if we do not have `Project` and `Window` between `Sort` and 
`Aggregate`.
```
== Parsed Logical Plan ==
'Sort [('key + 1) ASC], true
+- 'Aggregate [('key + 1)], [unresolvedalias('MAX('value),None)]
   +- 'UnresolvedRelation `src`, None
```
When that happens, we need to treat it as an aggregated function. The plan 
will be like:
```
Project [_c0#50]
+- Sort [aggOrder#51 ASC], true
   +- Aggregate [(key#45 + 1)], 
[(max(value#46),mode=Complete,isDistinct=false) AS _c0#50,(key#45 + 1) AS 
aggOrder#51]
  +- Subquery src
 +- Project [_1#43 AS key#45,_2#44 AS value#46]
+- LocalRelation [_1#43,_2#44], [[1,1],[-1,1]]
```

Let me show another case that does not need to treat `key+1` as an 
aggregate function. 
```SQL
SELECT value FROM src ORDER BY key + 1
```
```
== Parsed Logical Plan ==
'Sort [('key + 1) ASC], true
+- 'Project [unresolvedalias('value,None)]
   +- 'UnresolvedRelation `src`, None
```
In this case, we just need to push `key` into `Project`. 
```
Project [value#46]
+- Sort [(key#45 + 1) ASC], true
   +- Project [value#46,key#45]
  +- Subquery src
 +- Project [_1#43 AS key#45,_2#44 AS value#46]
+- LocalRelation [_1#43,_2#44], [[1,1],[-1,1]]
```


---
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: [SPARK-13379] [MLlib] Fix MLlib LogisticRegres...

2016-02-21 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-186998813
  
**[Test build #51658 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51658/consoleFull)**
 for PR 11298 at commit 
[`50071e2`](https://github.com/apache/spark/commit/50071e2cf9c76fd1919c437500b82bfd287b5f95).


---
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: [SPARK-13379] [MLlib] Fix MLlib LogisticRegres...

2016-02-21 Thread dbtsai
Github user dbtsai commented on the pull request:

https://github.com/apache/spark/pull/11258#issuecomment-186998722
  
LGTM. Merged into 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 pull request: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11260#discussion_r53585015
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 ---
@@ -260,6 +260,20 @@ case class Join(
 condition: Option[Expression])
   extends BinaryNode with PredicateHelper {
 
+  private def isBrocastHint(plan: LogicalPlan): Boolean = {
--- End diff --

OK. I've made it as variables to set & copy from other Join operator. 
Please take a look if this update is good. 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 pull request: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread gatorsmile
Github user gatorsmile commented on the pull request:

https://github.com/apache/spark/pull/11298#issuecomment-186996652
  
This PR is an incremental fix based on the previous solution for resolving 
missing attributes in Sort. This is not a clean fix, I like. However, to do a 
clean fix, we need to rewrite a lot of code in `ResolveSortReferences`. I am 
not sure if we should do it. Will explain the issues I hit when resolving this 
one. 


---
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-186996654
  
**[Test build #51655 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51655/consoleFull)**
 for PR 11296 at commit 
[`fb4d68a`](https://github.com/apache/spark/commit/fb4d68a1676dd31d79dbd5c7c074c1d9a09e103a).


---
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: [SPARK-13428] [SQL] Pushing Down Aggregate Exp...

2016-02-21 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

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

[SPARK-13428] [SQL] Pushing Down Aggregate Expressions in Sort into 
Aggregate

## What changes were proposed in this pull request?
When there exists the other operators between Sort and Aggregate, we are 
unable to push down the aggregate expressions in `Sort` into `Aggregate`. For 
example, in the following query, 

```SQL
select area, sum(product) over () as c from windowData
where product > 3 group by area, product
having avg(month) > 0 order by area, avg(month), product
```
The parsed logical plan is like
```
'Sort ['area ASC,'avg('month) ASC,'product ASC], true
+- 'Filter cast(('avg('month) > 0) as boolean)
   +- 'Aggregate ['area,'product], 
[unresolvedalias('area,None),unresolvedalias('sum('product) 
windowspecdefinition(UnspecifiedFrame) AS c#3,None)]
  +- 'Filter ('product > 3)
 +- 'UnresolvedRelation `windowData`, None
```
## How was the this patch tested?
Turn on a test case that `test("window function: Pushing aggregate 
Expressions in Sort to Aggregate")` exposes this issue.

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

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

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

https://github.com/apache/spark/pull/11298.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 #11298


commit 50071e2cf9c76fd1919c437500b82bfd287b5f95
Author: gatorsmile 
Date:   2016-02-22T03:57:45Z

pushing down aggregate expressions in Sort into Aggregate 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 pull request: [SPARK-13080] [SQL] Implement new Catalog API ...

2016-02-21 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/11293#issuecomment-186995812
  
I didn't review the core parts of this PR yet, hopefully @rxin had done 
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: [SPARK-13389] [SparkR] SparkR support first/la...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11267#issuecomment-186995268
  
**[Test build #51656 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51656/consoleFull)**
 for PR 11267 at commit 
[`6e78f7c`](https://github.com/apache/spark/commit/6e78f7c4a6145a5be57b1133555b8a4172d90129).


---
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: [SPARK-13379] [MLlib] Fix MLlib LogisticRegres...

2016-02-21 Thread dbtsai
Github user dbtsai commented on the pull request:

https://github.com/apache/spark/pull/11258#issuecomment-186994891
  
The default value in R's GLMNET is `1E-7`, and the default value in 
original LBFGS implementation is `1E-8`. In order to provide better and 
consistent result, let's change it to `1E-6` in a separate 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 pull request: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread jerryshao
Github user jerryshao commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-186994352
  
Jenkins, 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: [SPARK-12864][YARN] initialize executorIdCount...

2016-02-21 Thread zhonghaihua
Github user zhonghaihua commented on the pull request:

https://github.com/apache/spark/pull/10794#issuecomment-186993839
  
Hi @andrewor14 , the reason of test failed seems `GitException`. Could you 
retest it ? Thanks a lot.


---
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-186993681
  
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11296#issuecomment-186993684
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51654/
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: [SPARK-13389] [SparkR] SparkR support first/la...

2016-02-21 Thread yanboliang
Github user yanboliang commented on a diff in the pull request:

https://github.com/apache/spark/pull/11267#discussion_r53583948
  
--- Diff: R/pkg/R/functions.R ---
@@ -536,15 +536,27 @@ setMethod("factorial",
 #'
 #' Aggregate function: returns the first value in a group.
 #'
+#' The function by default returns the first values it sees. It will 
return the first non-NA
+#' value it sees when ignoreNAs is set to true. If all values are NA, then 
NA is returned.
+#'
 #' @rdname first
 #' @name first
 #' @family agg_funcs
 #' @export
-#' @examples \dontrun{first(df$c)}
+#' @examples
+#' \dontrun{
+#' first(df$c)
+#' first(df$c, TRUE)
+#' }
 setMethod("first",
-  signature(x = "Column"),
-  function(x) {
-jc <- callJStatic("org.apache.spark.sql.functions", "first", 
x@jc)
+  signature(x = "characterOrColumn"),
+  function(x, ignoreNAs = FALSE) {
--- End diff --

Make sense, I will update. 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 pull request: [SPARK-13389] [SparkR] SparkR support first/la...

2016-02-21 Thread yanboliang
Github user yanboliang commented on the pull request:

https://github.com/apache/spark/pull/11267#issuecomment-186993289
  
@shivaram Several R packages have similar function such as 
[first](http://www.inside-r.org/packages/cran/pastecs/docs/first), 
[aggregate](http://www.inside-r.org/r-doc/stats/aggregate), etc. Users need 
this feature in R and did some 
[workaround](http://stackoverflow.com/questions/28799753/how-to-select-the-last-one-test-without-na-in-r).
 


---
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: [SPARK-13220][Core]deprecate yarn-client and y...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11229#issuecomment-186993223
  
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 pull request: [SPARK-13220][Core]deprecate yarn-client and y...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11229#issuecomment-186993224
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51650/
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 pull request: [SPARK-12864][YARN] initialize executorIdCount...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10794#issuecomment-186993136
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51653/
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: [SPARK-12864][YARN] initialize executorIdCount...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/10794#issuecomment-186993134
  
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: [SPARK-13220][Core]deprecate yarn-client and y...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11229#issuecomment-186992965
  
**[Test build #51650 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51650/consoleFull)**
 for PR 11229 at commit 
[`330b0c4`](https://github.com/apache/spark/commit/330b0c44f5b42bb5615b4dcd37d86507b7ea8b6e).
 * 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: [SPARK-13427][SQL] Support USING clause in JOI...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11297#issuecomment-186992843
  
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 pull request: [SPARK-13427][SQL] Support USING clause in JOI...

2016-02-21 Thread dilipbiswal
GitHub user dilipbiswal opened a pull request:

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

[SPARK-13427][SQL] Support USING clause in JOIN.

## What changes were proposed in this pull request?

Support queries that JOIN tables with USING clause.
SELECT * from table1 JOIN table2 USING 


## How was the this patch tested?

Have added unit tests in SQLQuerySuite, CatalystQlSuite, 
ResolveNaturalJoinSuite

(If this patch involves UI changes, please attach a screenshot; otherwise, 
remove this)




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

$ git pull https://github.com/dilipbiswal/spark spark-13427

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

https://github.com/apache/spark/pull/11297.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 #11297


commit bdfe5ed32c3a1fa148b467723018de9746a26d73
Author: Dilip Biswal 
Date:   2016-02-19T23:23:07Z

[SPARK-13427] Support USING clause in JOIN.




---
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: [SPARK-13426][Core] Remove the support of SIMR

2016-02-21 Thread jerryshao
GitHub user jerryshao opened a pull request:

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

[SPARK-13426][Core] Remove the support of SIMR

## What changes were proposed in this pull request?

This PR removes the support of SIMR, since SIMR is not actively used and 
maintained for a long time, also is not supported from `SparkSubmit`, so here 
propose to remove it.

## How was the this patch tested?

This patch is tested locally by running unit tests.




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

$ git pull https://github.com/jerryshao/apache-spark SPARK-13426

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

https://github.com/apache/spark/pull/11296.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 #11296


commit fb4d68a1676dd31d79dbd5c7c074c1d9a09e103a
Author: jerryshao 
Date:   2016-02-22T03:16:41Z

Remove the support of SIMR




---
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: [SPARK-13381][SQL] Support for loading CSV wit...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11262#issuecomment-186986677
  
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 pull request: [SPARK-13381][SQL] Support for loading CSV wit...

2016-02-21 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/11262#issuecomment-186986680
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/51651/
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 pull request: [SPARK-13381][SQL] Support for loading CSV wit...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11262#issuecomment-186986183
  
**[Test build #51651 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51651/consoleFull)**
 for PR 11262 at commit 
[`3a39625`](https://github.com/apache/spark/commit/3a3962590890a9dbb9f65ec2115f495ba242fe8e).
 * 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: [MINOR][DOCS] Fix typos in ML doc

2016-02-21 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the pull request:

https://github.com/apache/spark/pull/11295#issuecomment-186980619
  
Thank you for your advice, @rxin.


---
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: [MINOR][DOCS] Fix typos in ML doc

2016-02-21 Thread dongjoon-hyun
Github user dongjoon-hyun closed the pull request at:

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


---
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: [SPARK-13381][SQL] Support for loading CSV wit...

2016-02-21 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-13321][SQL] Support nested UNION in par...

2016-02-21 Thread asfgit
Github user asfgit closed the pull request at:

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


---
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: [SPARK-13381][SQL] Support for loading CSV wit...

2016-02-21 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11262#issuecomment-186980335
  
Thanks - merging in 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: [SPARK-13321][SQL] Support nested UNION in par...

2016-02-21 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11204#issuecomment-186980183
  
Thanks - merging in 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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/11260#discussion_r53582665
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 ---
@@ -260,6 +260,20 @@ case class Join(
 condition: Option[Expression])
   extends BinaryNode with PredicateHelper {
 
+  private def isBrocastHint(plan: LogicalPlan): Boolean = {
--- End diff --

well you don't need to make the hint a parameter of the case class, but 
just some field you can set ...



---
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: [MINOR][DOCS] Fix typos in ML doc

2016-02-21 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/11295#issuecomment-186980067
  
Sounds good.



---
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: [MINOR][DOCS] Fix typos in ML doc

2016-02-21 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the pull request:

https://github.com/apache/spark/pull/11295#issuecomment-186979478
  
Oh, sure! Then, I will close this and make another PR after reviewing all 
md files in docs. Is it okay for you?


---
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: [SPARK-13233][SQL][WIP] Python Dataset

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7#issuecomment-186976616
  
**[Test build #51652 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/51652/consoleFull)**
 for PR 7 at commit 
[`8c32d31`](https://github.com/apache/spark/commit/8c32d311b7189b9a7a227a89d84681e97d2997a3).


---
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: [SPARK-13249][SQL] Add Filter checking nullabi...

2016-02-21 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/11235#issuecomment-186975972
  
cc @davies @marmbrus @liancheng @rxin 


---
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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11260#discussion_r53581769
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 ---
@@ -260,6 +260,20 @@ case class Join(
 condition: Option[Expression])
   extends BinaryNode with PredicateHelper {
 
+  private def isBrocastHint(plan: LogicalPlan): Boolean = {
--- End diff --

Is it ok for you? If so, I will make the change as 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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11260#discussion_r53581679
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 ---
@@ -260,6 +260,20 @@ case class Join(
 condition: Option[Expression])
   extends BinaryNode with PredicateHelper {
 
+  private def isBrocastHint(plan: LogicalPlan): Boolean = {
--- End diff --

E.g. each `case Join(left, right, joinType, condition) =>` would need to 
change to `case Join(left, right, joinType, condition, leftBroadcastHint, 
rightBroadcastHint)`.


---
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: [SPARK-13383][SQL] Keep broadcast hint after c...

2016-02-21 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/11260#discussion_r53581595
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/basicOperators.scala
 ---
@@ -260,6 +260,20 @@ case class Join(
 condition: Option[Expression])
   extends BinaryNode with PredicateHelper {
 
+  private def isBrocastHint(plan: LogicalPlan): Boolean = {
--- End diff --

yeah, I see. But as we need to add two broadcast hint for left and right 
plans. Considering Join is broadly used in many places as pattern matching 
usage, I am afraid that we need to change too many places for this change.


---
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: [SPARK-13321][SQL] Support nested UNION in par...

2016-02-21 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/11204#issuecomment-186974334
  
cc @rxin 


---
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: [SPARK-13363][SQL] Support Aggregator in DataF...

2016-02-21 Thread maropu
Github user maropu closed the pull request at:

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


---
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: [SPARK-13381][SQL] Support for loading CSV wit...

2016-02-21 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/11262#issuecomment-186973763
  
**[Test build #2563 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/2563/consoleFull)**
 for PR 11262 at commit 
[`3a39625`](https://github.com/apache/spark/commit/3a3962590890a9dbb9f65ec2115f495ba242fe8e).
 * 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



  1   2   3   4   5   >