[GitHub] spark pull request #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79330219
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -0,0 +1,159 @@
+/*
+ * 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.command
+
+import scala.collection.mutable
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
CatalogTable}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, 
BasicColStats, Statistics}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types._
+
+
+/**
+ * Analyzes the given columns of the given table in the current database 
to generate statistics,
+ * which will be used in query optimizations.
+ */
+case class AnalyzeColumnCommand(
+tableIdent: TableIdentifier,
+columnNames: Seq[String]) extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val sessionState = sparkSession.sessionState
+val relation = 
EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent))
+
+// check correctness of column names
+val validColumns = mutable.MutableList[NamedExpression]()
+val resolver = sessionState.conf.resolver
+columnNames.foreach { col =>
+  val exprOption = relation.resolve(col.split("\\."), resolver)
+  if (exprOption.isEmpty) {
+throw new AnalysisException(s"Invalid column name: $col")
+  }
+  if (validColumns.map(_.exprId).contains(exprOption.get.exprId)) {
+throw new AnalysisException(s"Duplicate column name: $col")
+  }
+  validColumns += exprOption.get
+}
+
+relation match {
+  case catalogRel: CatalogRelation =>
+updateStats(catalogRel.catalogTable,
+  AnalyzeTableCommand.calculateTotalSize(sparkSession, 
catalogRel.catalogTable))
+
+  case logicalRel: LogicalRelation if 
logicalRel.catalogTable.isDefined =>
+updateStats(logicalRel.catalogTable.get, 
logicalRel.relation.sizeInBytes)
+
+  case otherRelation =>
+throw new AnalysisException("ANALYZE TABLE is not supported for " +
+  s"${otherRelation.nodeName}.")
+}
+
+def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit 
= {
+  // Collect statistics per column.
+  // The first element in the result will be the overall row count, 
the following elements
+  // will be structs containing all column stats.
+  // The layout of each struct follows the layout of the BasicColStats.
+  val ndvMaxErr = sessionState.conf.ndvMaxError
+  val expressions = Count(Literal(1)).toAggregateExpression() +:
+validColumns.map(ColumnStatsStruct(_, ndvMaxErr))
+  val namedExpressions = expressions.map(e => Alias(e, e.toString)())
+  val statsRow = Dataset.ofRows(sparkSession, Aggregate(Nil, 
namedExpressions, relation))
+.queryExecution.toRdd.collect().head
+
+  // unwrap the result
+  val rowCount = statsRow.getLong(0)
+  val colStats = validColumns.zipWithIndex.map { case (expr, i) =>
+val colInfo = statsRow.getStruct(i + 1, 
ColumnStatsStruct.statsNumber)
+val colStats = ColumnStatsStruct.unwrapRow(expr, colInfo)
+(expr.name, colStats)
+  }.toMap
+
+  val statistics =
+Statistics(sizeInBytes = newTotalSize, rowCount = Some(rowCount), 
basicColStats = colStats)
+  sessionState.catalog.alterTable(catalogTable.copy(stats = 
Some(statistics)))
+

[GitHub] spark pull request #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79329962
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -0,0 +1,209 @@
+/*
+ * 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.command
+
+import scala.collection.mutable
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
CatalogTable}
+import org.apache.spark.sql.catalyst.plans.logical.{BasicColStats, 
Statistics}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.types._
+
+
+/**
+ * Analyzes the given columns of the given table in the current database 
to generate statistics,
+ * which will be used in query optimizations.
+ */
+case class AnalyzeColumnCommand(
+tableName: String,
+columnNames: Seq[String]) extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val sessionState = sparkSession.sessionState
+val tableIdent = sessionState.sqlParser.parseTableIdentifier(tableName)
+val relation = 
EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent))
+
+// check correctness for column names
+val attributeNames = relation.output.map(_.name.toLowerCase)
+val invalidColumns = columnNames.filterNot { col => 
attributeNames.contains(col.toLowerCase)}
+if (invalidColumns.nonEmpty) {
+  throw new AnalysisException(s"Invalid columns for table $tableName: 
$invalidColumns.")
+}
+
+relation match {
+  case catalogRel: CatalogRelation =>
+updateStats(catalogRel.catalogTable,
+  AnalyzeTableCommand.calculateTotalSize(sparkSession, 
catalogRel.catalogTable))
+
+  case logicalRel: LogicalRelation if 
logicalRel.catalogTable.isDefined =>
+updateStats(logicalRel.catalogTable.get, 
logicalRel.relation.sizeInBytes)
+
+  case otherRelation =>
+throw new AnalysisException(s"ANALYZE TABLE is not supported for " 
+
+  s"${otherRelation.nodeName}.")
+}
+
+def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit 
= {
--- End diff --

Can this be then defined outside? I know this is about personal taste but 
it causes confusion to at least two of reviewers here.


---
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 #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79329866
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -0,0 +1,159 @@
+/*
+ * 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.command
+
+import scala.collection.mutable
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
CatalogTable}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, 
BasicColStats, Statistics}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types._
+
+
+/**
+ * Analyzes the given columns of the given table in the current database 
to generate statistics,
+ * which will be used in query optimizations.
+ */
+case class AnalyzeColumnCommand(
+tableIdent: TableIdentifier,
+columnNames: Seq[String]) extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val sessionState = sparkSession.sessionState
+val relation = 
EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent))
+
+// check correctness of column names
+val validColumns = mutable.MutableList[NamedExpression]()
+val resolver = sessionState.conf.resolver
+columnNames.foreach { col =>
+  val exprOption = relation.resolve(col.split("\\."), resolver)
+  if (exprOption.isEmpty) {
+throw new AnalysisException(s"Invalid column name: $col")
+  }
+  if (validColumns.map(_.exprId).contains(exprOption.get.exprId)) {
+throw new AnalysisException(s"Duplicate column name: $col")
+  }
+  validColumns += exprOption.get
+}
+
+relation match {
+  case catalogRel: CatalogRelation =>
+updateStats(catalogRel.catalogTable,
+  AnalyzeTableCommand.calculateTotalSize(sparkSession, 
catalogRel.catalogTable))
+
+  case logicalRel: LogicalRelation if 
logicalRel.catalogTable.isDefined =>
+updateStats(logicalRel.catalogTable.get, 
logicalRel.relation.sizeInBytes)
+
+  case otherRelation =>
+throw new AnalysisException("ANALYZE TABLE is not supported for " +
+  s"${otherRelation.nodeName}.")
+}
+
+def updateStats(catalogTable: CatalogTable, newTotalSize: Long): Unit 
= {
+  // Collect statistics per column.
+  // The first element in the result will be the overall row count, 
the following elements
+  // will be structs containing all column stats.
+  // The layout of each struct follows the layout of the BasicColStats.
+  val ndvMaxErr = sessionState.conf.ndvMaxError
+  val expressions = Count(Literal(1)).toAggregateExpression() +:
+validColumns.map(ColumnStatsStruct(_, ndvMaxErr))
+  val namedExpressions = expressions.map(e => Alias(e, e.toString)())
+  val statsRow = Dataset.ofRows(sparkSession, Aggregate(Nil, 
namedExpressions, relation))
+.queryExecution.toRdd.collect().head
+
+  // unwrap the result
+  val rowCount = statsRow.getLong(0)
+  val colStats = validColumns.zipWithIndex.map { case (expr, i) =>
+val colInfo = statsRow.getStruct(i + 1, 
ColumnStatsStruct.statsNumber)
+val colStats = ColumnStatsStruct.unwrapRow(expr, colInfo)
+(expr.name, colStats)
+  }.toMap
+
+  val statistics =
+Statistics(sizeInBytes = newTotalSize, rowCount = Some(rowCount), 
basicColStats = colStats)
+  sessionState.catalog.alterTable(catalogTable.copy(stats = 
Some(statistics)))
   

[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15146
  
**[Test build #65588 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65588/consoleFull)**
 for PR 15146 at commit 
[`2f8e79f`](https://github.com/apache/spark/commit/2f8e79f81e19889b27ce762cd4c832c71fff4613).
 * 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 #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79329639
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala ---
@@ -101,4 +101,47 @@ class StatisticsSuite extends QueryTest with 
SharedSQLContext {
   checkTableStats(tableName, expectedRowCount = Some(2))
 }
   }
+
+  test("test column-level statistics for data source table created in 
InMemoryCatalog") {
+def checkColStats(colStats: BasicColStats, expectedColStats: 
BasicColStats): Unit = {
+  assert(colStats.dataType == expectedColStats.dataType)
+  assert(colStats.numNulls == expectedColStats.numNulls)
+  assert(colStats.max == expectedColStats.max)
+  assert(colStats.min == expectedColStats.min)
+  if (expectedColStats.ndv.isDefined) {
+// ndv is an approximate value, so we just make sure we have the 
value
+assert(colStats.ndv.get >= 0)
--- End diff --

Why 3 standard deviations? I think for tests in this suite, we just need to 
make sure we get the stats, we should leave the accuracy test to 
HyperLogLogPlusPlusSuite, right?


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

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



[GitHub] spark pull request #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79329457
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala
 ---
@@ -32,19 +34,70 @@ package org.apache.spark.sql.catalyst.plans.logical
  * @param sizeInBytes Physical size in bytes. For leaf operators this 
defaults to 1, otherwise it
  *defaults to the product of children's `sizeInBytes`.
  * @param rowCount Estimated number of rows.
+ * @param basicColStats Basic column-level statistics.
  * @param isBroadcastable If true, output is small enough to be used in a 
broadcast join.
  */
 case class Statistics(
 sizeInBytes: BigInt,
 rowCount: Option[BigInt] = None,
+basicColStats: Map[String, BasicColStats] = Map.empty,
 isBroadcastable: Boolean = false) {
+
   override def toString: String = "Statistics(" + simpleString + ")"
 
   /** Readable string representation for the Statistics. */
   def simpleString: String = {
 Seq(s"sizeInBytes=$sizeInBytes",
   if (rowCount.isDefined) s"rowCount=${rowCount.get}" else "",
+  if (basicColStats.nonEmpty) s"basicColStats=$basicColStats" else "",
   s"isBroadcastable=$isBroadcastable"
-).filter(_.nonEmpty).mkString("", ", ", "")
+).filter(_.nonEmpty).mkString(", ")
+  }
+}
+
+case class BasicColStats(
--- End diff --

@viirya thanks for the advice.


---
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 #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79329382
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -0,0 +1,159 @@
+/*
+ * 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.command
+
+import scala.collection.mutable
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
CatalogTable}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, 
BasicColStats, Statistics}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types._
+
+
+/**
+ * Analyzes the given columns of the given table in the current database 
to generate statistics,
+ * which will be used in query optimizations.
+ */
+case class AnalyzeColumnCommand(
+tableIdent: TableIdentifier,
+columnNames: Seq[String]) extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val sessionState = sparkSession.sessionState
+val relation = 
EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent))
+
+// check correctness of column names
+val validColumns = mutable.MutableList[NamedExpression]()
+val resolver = sessionState.conf.resolver
+columnNames.foreach { col =>
+  val exprOption = relation.resolve(col.split("\\."), resolver)
--- End diff --

I think I'll check the column by hand here based on the caseSensitive 
configuration, so that I can use the name in relation's output. Using 
`resolve()` has a problem, for example:
`create table (key int)`
`analyze ... for columns KEY`
then if using `resolve()`,the name of expression is `KEY`(case 
insensitive), so will be the name in `colStats` in Statistics class.
As a result, when we load the stats from metastore, the name in schema is 
`key`, while the name in table parameter is `KEY`, they don't match, so we lost 
the column stats.


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15053
  
@holdenk I am also cautious but leaving everything but adding `df.show()` 
in the package docstring with cleaning up duplicated defining dataframes in 
each docstring will be minimal change   and fine for the goal.


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

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



[GitHub] spark issue #15054: [SPARK-17502] [SQL] Fix Multiple Bugs in DDL Statements ...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15054: [SPARK-17502] [SQL] Fix Multiple Bugs in DDL Statements ...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15054: [SPARK-17502] [SQL] Fix Multiple Bugs in DDL Statements ...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #15024: [SPARK-17470][SQL] unify path for data source table and ...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15024
  
For `FileFormat `, [`allPaths` is changed to `paths ++ new 
CaseInsensitiveMap(options).get("path")`](https://github.com/cloud-fan/spark/blob/9ab4b8ce3dd7c41edb0681ff903d218bad2e4225/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L331).
 How about the external data source that extend `CreatableRelationProvider`? Do 
we need to make the same change at 
[here](https://github.com/cloud-fan/spark/blob/9ab4b8ce3dd7c41edb0681ff903d218bad2e4225/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala#L325)?




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

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



[GitHub] spark issue #15024: [SPARK-17470][SQL] unify path for data source table and ...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15024
  
It sounds like we also need to call `optionsToStorageFormat` for 
`visitCreateTempViewUsing`. 


---
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 #15024: [SPARK-17470][SQL] unify path for data source tab...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15024#discussion_r79328309
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSource.scala
 ---
@@ -507,3 +400,117 @@ case class DataSource(
 }
   }
 }
+
+object DataSource {
+  /** A map to maintain backward compatibility in case we move data 
sources around. */
+  private val backwardCompatibilityMap: Map[String, String] = {
+val jdbc = classOf[JdbcRelationProvider].getCanonicalName
+val json = classOf[JsonFileFormat].getCanonicalName
+val parquet = classOf[ParquetFileFormat].getCanonicalName
+val csv = classOf[CSVFileFormat].getCanonicalName
+val libsvm = "org.apache.spark.ml.source.libsvm.LibSVMFileFormat"
+val orc = "org.apache.spark.sql.hive.orc.OrcFileFormat"
+
+Map(
+  "org.apache.spark.sql.jdbc" -> jdbc,
+  "org.apache.spark.sql.jdbc.DefaultSource" -> jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc.DefaultSource" -> 
jdbc,
+  "org.apache.spark.sql.execution.datasources.jdbc" -> jdbc,
+  "org.apache.spark.sql.json" -> json,
+  "org.apache.spark.sql.json.DefaultSource" -> json,
+  "org.apache.spark.sql.execution.datasources.json" -> json,
+  "org.apache.spark.sql.execution.datasources.json.DefaultSource" -> 
json,
+  "org.apache.spark.sql.parquet" -> parquet,
+  "org.apache.spark.sql.parquet.DefaultSource" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet" -> parquet,
+  "org.apache.spark.sql.execution.datasources.parquet.DefaultSource" 
-> parquet,
+  "org.apache.spark.sql.hive.orc.DefaultSource" -> orc,
+  "org.apache.spark.sql.hive.orc" -> orc,
+  "org.apache.spark.ml.source.libsvm.DefaultSource" -> libsvm,
+  "org.apache.spark.ml.source.libsvm" -> libsvm,
+  "com.databricks.spark.csv" -> csv
+)
+  }
+
+  /**
+   * Class that were removed in Spark 2.0. Used to detect incompatibility 
libraries for Spark 2.0.
+   */
+  private val spark2RemovedClasses = Set(
+"org.apache.spark.sql.DataFrame",
+"org.apache.spark.sql.sources.HadoopFsRelationProvider",
+"org.apache.spark.Logging")
+
+  /** Given a provider name, look up the data source class definition. */
+  def lookupDataSource(provider0: String): Class[_] = {
+val provider = backwardCompatibilityMap.getOrElse(provider0, provider0)
+val provider2 = s"$provider.DefaultSource"
+val loader = Utils.getContextOrSparkClassLoader
+val serviceLoader = ServiceLoader.load(classOf[DataSourceRegister], 
loader)
+
+try {
+  
serviceLoader.asScala.filter(_.shortName().equalsIgnoreCase(provider)).toList 
match {
+// the provider format did not match any given registered aliases
+case Nil =>
+  try {
+
Try(loader.loadClass(provider)).orElse(Try(loader.loadClass(provider2))) match {
+  case Success(dataSource) =>
+// Found the data source using fully qualified path
+dataSource
+  case Failure(error) =>
+if (provider.toLowerCase == "orc" ||
+  provider.startsWith("org.apache.spark.sql.hive.orc")) {
+  throw new AnalysisException(
+"The ORC data source must be used with Hive support 
enabled")
+} else if (provider.toLowerCase == "avro" ||
+  provider == "com.databricks.spark.avro") {
+  throw new AnalysisException(
+s"Failed to find data source: ${provider.toLowerCase}. 
Please find an Avro " +
+  "package at " +
+  
"https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects;)
+} else {
+  throw new ClassNotFoundException(
+s"Failed to find data source: $provider. Please find 
packages at " +
+  
"https://cwiki.apache.org/confluence/display/SPARK/Third+Party+Projects;,
+error)
+}
+}
+  } catch {
+case e: NoClassDefFoundError => // This one won't be caught by 
Scala NonFatal
+  // NoClassDefFoundError's class name uses "/" rather than 
"." for packages
+  val className = e.getMessage.replaceAll("/", ".")
+  if (spark2RemovedClasses.contains(className)) {
+throw new ClassNotFoundException(s"$className was removed 
in Spark 2.0. " +
+  "Please check if your library is compatible with Spark 
2.0", e)
+  } else {
+

[GitHub] spark pull request #13599: [SPARK-13587] [PYSPARK] Support virtualenv in pys...

2016-09-18 Thread zjffdu
Github user zjffdu commented on a diff in the pull request:

https://github.com/apache/spark/pull/13599#discussion_r79328285
  
--- Diff: 
core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala ---
@@ -69,6 +84,67 @@ private[spark] class PythonWorkerFactory(pythonExec: 
String, envVars: Map[String
   }
 
   /**
+   * Create virtualenv using native virtualenv or conda
+   *
+   * Native Virtualenv:
+   *   -  Execute command: virtualenv -p pythonExec --no-site-packages 
virtualenvName
+   *   -  Execute command: python -m pip --cache-dir cache-dir install -r 
requirement_file
+   *
+   * Conda
+   *   -  Execute command: conda create --prefix prefix --file 
requirement_file -y
+   *
+   */
+  def setupVirtualEnv(): Unit = {
+logDebug("Start to setup virtualenv...")
+logDebug("user.dir=" + System.getProperty("user.dir"))
+logDebug("user.home=" + System.getProperty("user.home"))
+
+require(virtualEnvType == "native" || virtualEnvType == "conda",
+  s"VirtualEnvType: ${virtualEnvType} is not supported" )
+virtualEnvName = "virtualenv_" + conf.getAppId + "_" + 
VIRTUALENV_ID.getAndIncrement()
+// use the absolute path when it is local mode otherwise just use 
filename as it would be
+// fetched from FileServer
+val pyspark_requirements =
+  if (Utils.isLocalMaster(conf)) {
+conf.get("spark.pyspark.virtualenv.requirements")
+  } else {
+conf.get("spark.pyspark.virtualenv.requirements").split("/").last
+  }
+
+val createEnvCommand =
+  if (virtualEnvType == "native") {
+Arrays.asList(virtualEnvPath,
+  "-p", pythonExec,
+  "--no-site-packages", virtualEnvName)
+  } else {
+Arrays.asList(virtualEnvPath,
+  "create", "--prefix", System.getProperty("user.dir") + "/" + 
virtualEnvName,
+  "--file", pyspark_requirements, "-y")
+  }
+execCommand(createEnvCommand)
+// virtualenv will be created in the working directory of Executor.
+virtualPythonExec = virtualEnvName + "/bin/python"
+if (virtualEnvType == "native") {
+  execCommand(Arrays.asList(virtualPythonExec, "-m", "pip",
+"--cache-dir", System.getProperty("user.home"),
+"install", "-r", pyspark_requirements))
+}
+  }
+
+  def execCommand(commands: java.util.List[String]): Unit = {
+logDebug("Running command:" + commands.asScala.mkString(" "))
+val pb = new ProcessBuilder(commands).inheritIO()
--- End diff --

It is for redirect the process output to executor output in case anything 
wrong when setting up the virtualenv.  Otherwise it is difficult to know what's 
wrong. I think it is acceptable to redirect output by default. 


---
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 #13599: [SPARK-13587] [PYSPARK] Support virtualenv in pys...

2016-09-18 Thread zjffdu
Github user zjffdu commented on a diff in the pull request:

https://github.com/apache/spark/pull/13599#discussion_r79328171
  
--- Diff: 
core/src/main/scala/org/apache/spark/api/python/PythonWorkerFactory.scala ---
@@ -69,6 +84,67 @@ private[spark] class PythonWorkerFactory(pythonExec: 
String, envVars: Map[String
   }
 
   /**
+   * Create virtualenv using native virtualenv or conda
+   *
+   * Native Virtualenv:
+   *   -  Execute command: virtualenv -p pythonExec --no-site-packages 
virtualenvName
+   *   -  Execute command: python -m pip --cache-dir cache-dir install -r 
requirement_file
+   *
+   * Conda
+   *   -  Execute command: conda create --prefix prefix --file 
requirement_file -y
+   *
+   */
+  def setupVirtualEnv(): Unit = {
+logDebug("Start to setup virtualenv...")
+logDebug("user.dir=" + System.getProperty("user.dir"))
+logDebug("user.home=" + System.getProperty("user.home"))
+
+require(virtualEnvType == "native" || virtualEnvType == "conda",
+  s"VirtualEnvType: ${virtualEnvType} is not supported" )
+virtualEnvName = "virtualenv_" + conf.getAppId + "_" + 
VIRTUALENV_ID.getAndIncrement()
+// use the absolute path when it is local mode otherwise just use 
filename as it would be
+// fetched from FileServer
+val pyspark_requirements =
+  if (Utils.isLocalMaster(conf)) {
+conf.get("spark.pyspark.virtualenv.requirements")
+  } else {
+conf.get("spark.pyspark.virtualenv.requirements").split("/").last
+  }
+
+val createEnvCommand =
+  if (virtualEnvType == "native") {
+Arrays.asList(virtualEnvPath,
+  "-p", pythonExec,
+  "--no-site-packages", virtualEnvName)
+  } else {
+Arrays.asList(virtualEnvPath,
+  "create", "--prefix", System.getProperty("user.dir") + "/" + 
virtualEnvName,
+  "--file", pyspark_requirements, "-y")
+  }
+execCommand(createEnvCommand)
+// virtualenv will be created in the working directory of Executor.
+virtualPythonExec = virtualEnvName + "/bin/python"
+if (virtualEnvType == "native") {
+  execCommand(Arrays.asList(virtualPythonExec, "-m", "pip",
+"--cache-dir", System.getProperty("user.home"),
+"install", "-r", pyspark_requirements))
+}
+  }
+
+  def execCommand(commands: java.util.List[String]): Unit = {
+logDebug("Running command:" + commands.asScala.mkString(" "))
+val pb = new ProcessBuilder(commands).inheritIO()
+pb.environment().putAll(envVars.asJava)
--- End diff --

Correct, it is not necessary here, will remove it. 


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread holdenk
Github user holdenk commented on the issue:

https://github.com/apache/spark/pull/15053
  
I was thinking that the user would probably read the package string 
documentation before looking at the individual functions (or if they went 
looking for the definition of the dataframe). I'm a little hesitant to add a 
bunch of copy and paste code so its possible I'm being overly cautious.


---
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 #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79327748
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/StatisticsSuite.scala ---
@@ -101,4 +101,47 @@ class StatisticsSuite extends QueryTest with 
SharedSQLContext {
   checkTableStats(tableName, expectedRowCount = Some(2))
 }
   }
+
+  test("test column-level statistics for data source table created in 
InMemoryCatalog") {
+def checkColStats(colStats: BasicColStats, expectedColStats: 
BasicColStats): Unit = {
+  assert(colStats.dataType == expectedColStats.dataType)
+  assert(colStats.numNulls == expectedColStats.numNulls)
+  assert(colStats.max == expectedColStats.max)
+  assert(colStats.min == expectedColStats.min)
+  if (expectedColStats.ndv.isDefined) {
+// ndv is an approximate value, so we just make sure we have the 
value
+assert(colStats.ndv.get >= 0)
--- End diff --

It should be:

1. estimate column statistics for several times.
2. calculate mean and standard deviation of ndv for the multiple statistics.
3. check if the real ndv value is within (mean + 3 * sd, mean - 3 * sd).

Please correct me if it is wrong.


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

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



[GitHub] spark issue #15131: [SPARK-17577][SparkR][Core] SparkR support add files to ...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15131
  
**[Test build #65590 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65590/consoleFull)**
 for PR 15131 at commit 
[`acfbd8a`](https://github.com/apache/spark/commit/acfbd8a8837449932bccbb5378ddb909188bfad8).


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

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



[GitHub] spark issue #15131: [SPARK-17577][SparkR][Core] SparkR support add files to ...

2016-09-18 Thread yanboliang
Github user yanboliang commented on the issue:

https://github.com/apache/spark/pull/15131
  
Jenkins, test this please.


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

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



[GitHub] spark issue #15102: [SPARK-17346][SQL] Add Kafka source for Structured Strea...

2016-09-18 Thread zsxwing
Github user zsxwing commented on the issue:

https://github.com/apache/spark/pull/15102
  
> We do need to handle it comparing completely different topicpartitions, 
because it's entirely possible to have a job with a single topicpartition A, 
which is deleted or unsubscribed, and then single topicpartition B is added, in 
the space of one batch.

Yeah, it's also possible that when a batch is running, a single 
topicpartition C is created then deleted. But the Kafka source doesn't even 
know C. The root issue is currently the approach is polling metadata, and any 
metadata changes between two pollings are missing.


---
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 #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79327304
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/logical/Statistics.scala
 ---
@@ -32,19 +34,70 @@ package org.apache.spark.sql.catalyst.plans.logical
  * @param sizeInBytes Physical size in bytes. For leaf operators this 
defaults to 1, otherwise it
  *defaults to the product of children's `sizeInBytes`.
  * @param rowCount Estimated number of rows.
+ * @param basicColStats Basic column-level statistics.
  * @param isBroadcastable If true, output is small enough to be used in a 
broadcast join.
  */
 case class Statistics(
 sizeInBytes: BigInt,
 rowCount: Option[BigInt] = None,
+basicColStats: Map[String, BasicColStats] = Map.empty,
 isBroadcastable: Boolean = false) {
+
   override def toString: String = "Statistics(" + simpleString + ")"
 
   /** Readable string representation for the Statistics. */
   def simpleString: String = {
 Seq(s"sizeInBytes=$sizeInBytes",
   if (rowCount.isDefined) s"rowCount=${rowCount.get}" else "",
+  if (basicColStats.nonEmpty) s"basicColStats=$basicColStats" else "",
   s"isBroadcastable=$isBroadcastable"
-).filter(_.nonEmpty).mkString("", ", ", "")
+).filter(_.nonEmpty).mkString(", ")
+  }
+}
+
+case class BasicColStats(
--- End diff --

Add a section of comments explaining the meaning for each statistic? E.g., 
Readers may not know what `ndv` means.


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

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



[GitHub] spark issue #14784: [SPARK-17210][SPARKR] sparkr.zip is not distributed to e...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14784: [SPARK-17210][SPARKR] sparkr.zip is not distributed to e...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14784: [SPARK-17210][SPARKR] sparkr.zip is not distributed to e...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #11105: [SPARK-12469][CORE] Data Property accumulators for Spark

2016-09-18 Thread holdenk
Github user holdenk commented on the issue:

https://github.com/apache/spark/pull/11105
  
If @rxin or @squito has the bandwith to continue reviewing I'd really 
appreciate it (especially on the mergeImpl / addImpl wrapping or if should go 
about it in another way).


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15146
  
**[Test build #65586 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65586/consoleFull)**
 for PR 15146 at commit 
[`394495d`](https://github.com/apache/spark/commit/394495d0effc0c217ff6bb0df92e664636e0dfa0).
 * 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 #15090: [SPARK-17073] [SQL] generate column-level statist...

2016-09-18 Thread wzhfy
Github user wzhfy commented on a diff in the pull request:

https://github.com/apache/spark/pull/15090#discussion_r79326407
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/AnalyzeColumnCommand.scala
 ---
@@ -0,0 +1,159 @@
+/*
+ * 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.command
+
+import scala.collection.mutable
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.EliminateSubqueryAliases
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
CatalogTable}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical.{Aggregate, 
BasicColStats, Statistics}
+import org.apache.spark.sql.execution.datasources.LogicalRelation
+import org.apache.spark.sql.types._
+
+
+/**
+ * Analyzes the given columns of the given table in the current database 
to generate statistics,
+ * which will be used in query optimizations.
+ */
+case class AnalyzeColumnCommand(
+tableIdent: TableIdentifier,
+columnNames: Seq[String]) extends RunnableCommand {
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+val sessionState = sparkSession.sessionState
+val relation = 
EliminateSubqueryAliases(sessionState.catalog.lookupRelation(tableIdent))
+
+// check correctness of column names
+val validColumns = mutable.MutableList[NamedExpression]()
+val resolver = sessionState.conf.resolver
+columnNames.foreach { col =>
+  val exprOption = relation.resolve(col.split("\\."), resolver)
--- End diff --

@hvanhovell Do you mean using 
`relation.resolve(UnresolvedAttribute.parseAttributeName(col), resolver)`? In 
this case, \`a.d\` will be split into `a` and `d`, because after parsing, 
\`a.d\` will become a.d in analyze command. I think `relation.resolve(Seq(col), 
resolver)` is ok, right?


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

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



[GitHub] spark issue #14784: [SPARK-17210][SPARKR] sparkr.zip is not distributed to e...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14784
  
**[Test build #65589 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65589/consoleFull)**
 for PR 14784 at commit 
[`e635164`](https://github.com/apache/spark/commit/e635164c6b25d36e5913384d9ab1151d2ed59a39).


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

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



[GitHub] spark issue #14784: [SPARK-17210][SPARKR] sparkr.zip is not distributed to e...

2016-09-18 Thread zjffdu
Github user zjffdu commented on the issue:

https://github.com/apache/spark/pull/14784
  
@shivaram @felixcheung Sorry for late response, I just rebase the PR and 
also take spark.master over master. Please help review.  


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15053
  
Oh, I meant just don't touch the `globs` in `_test()` but just print the 
global dataframes (which should be rather `show()` to show the contents) so 
that users can understand the input and output. So, my scenario is, users open 
the doc and see the input dataframes first and then check the function 
documentation they want.

I do like the idea of defining each within each docstring but maybe we 
could try the minimal change suggestion to reach the same goal.

I guess she meant defining the global dataframes within the package 
docstring which is, for example,

```python
"""
# here
"""

import doctest
import sys
...
```

However, after investigating, it seems we can't easily do that. So, I 
suggested to print them in the package docstring such as

```python
"""
>>> df.show()
+-+
|field|
+-+
|value|
|value|
+-+
"""

import doctest
import sys
...
```

rather than defining them in the package docstring.




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

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



[GitHub] spark issue #14597: [SPARK-17017][MLLIB][ML] add a chiSquare Selector based ...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14597: [SPARK-17017][MLLIB][ML] add a chiSquare Selector based ...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14597: [SPARK-17017][MLLIB][ML] add a chiSquare Selector based ...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14597
  
**[Test build #65585 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65585/consoleFull)**
 for PR 14597 at commit 
[`88d2143`](https://github.com/apache/spark/commit/88d2143989a4219020100999c53bb1186fce5d1c).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/15146
  
cc @hvanhovell @cloud-fan 


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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once and a...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15146
  
**[Test build #65588 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65588/consoleFull)**
 for PR 15146 at commit 
[`2f8e79f`](https://github.com/apache/spark/commit/2f8e79f81e19889b27ce762cd4c832c71fff4613).


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread mortada
Github user mortada commented on the issue:

https://github.com/apache/spark/pull/15053
  
@HyukjinKwon I understand we can have `py.test` and `doctest`, but I don't 
quite see how we could define the input DataFrame globally while at the same 
time have a clear, self-contained docstring for each function?

@holdenk could you please elaborate on what you mean? 

If we want to repeat something like this in every docstring
```
>>> print(df.collect())
```
we might as well simply include how to actually create the DataFrame so the 
user can easily reproduce the example?

It seems to me that the user would often want to see the docstring to 
understand how a function works, and they may not be looking at some global 
documentation as a whole. And the fact that many of the input DataFrames are 
the same is really just a convenience for the doc writer and not a requirement.

For instance this is the docstring for a numpy method (`numpy.argmax`), and 
the example is with the input clearly defined:
```
Examples

>>> a = np.arange(6).reshape(2,3)
>>> a
array([[0, 1, 2],
   [3, 4, 5]])
>>> np.argmax(a)
5
>>> np.argmax(a, axis=0)
array([1, 1, 1])
>>> np.argmax(a, axis=1)
array([2, 2])
```

IMHO it seems odd to require the user to look at some global doc in order 
to follow the example usage for one single function


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

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



[GitHub] spark issue #14995: [Test Only][SPARK-6235][CORE]Address various 2G limits

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14995: [Test Only][SPARK-6235][CORE]Address various 2G limits

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14995: [Test Only][SPARK-6235][CORE]Address various 2G limits

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14995
  
**[Test build #65584 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65584/consoleFull)**
 for PR 14995 at commit 
[`57833e3`](https://github.com/apache/spark/commit/57833e3a2b315294a1a8384d3fcee3926d4c55ce).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #15041: [SPARK-17488][CORE] TakeAndOrder will OOM when the data ...

2016-09-18 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/15041
  
cc @cloud-fan @hvanhovell 


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

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



[GitHub] spark issue #15054: [SPARK-17502] [SQL] Fix Multiple Bugs in DDL Statements ...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15054
  
**[Test build #65587 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65587/consoleFull)**
 for PR 15054 at commit 
[`f305c4c`](https://github.com/apache/spark/commit/f305c4c2ae1bfb2c4d94d1e17942ff6273c3da2b).


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

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



[GitHub] spark issue #15147: [SPARK-17545] [SQL] Handle additional time offset format...

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15147
  
FYI, we backported this to branch 2.0 too. So this will be fixed from 2.0.1 
https://github.com/apache/spark/pull/14799.


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

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



[GitHub] spark issue #15147: [SPARK-17545] [SQL] Handle additional time offset format...

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15147
  
cc @srowen who was in the JIRA too.


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

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



[GitHub] spark issue #15147: [SPARK-17545] [SQL] Handle additional time offset format...

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15147
  
To continue the discussion of JIRA, I think the issue you faced is to read 
those in CSV?

Whether it is intended or not in `FastDateFormat`, the default pattern 
`"-MM-dd'T'HH:mm:ss.SSSZZ"` covers this.

it seems I can't reproduce the issue you met in the JIRA. Have you tried 
the problematic codes in the master branch? 

That would not ran in Spark 2.0 but we made a change.

```scala
val path = "/tmp/timestamps"
val textDf = Seq(
  "time",
  "2015-07-20T15:09:23.736-0500",
  "2015-07-20T15:10:51.687-0500",
  "2015-11-21T23:15:01.499-0600").toDF()
textDf.coalesce(1).write.text(path)
val df = spark.read.format("csv")
  .option("header", "true")
  .option("inferSchema", "true")
  .load(path)
df.show()
df.printSchema()
```

works fine

```
++
|time|
++
|2015-07-20 13:09:...|
|2015-07-20 13:10:...|
|2015-11-21 21:15:...|
++

root
 |-- time: timestamp (nullable = true)

```
 


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

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



[GitHub] spark issue #15147: [SPARK-17545] [SQL] Handle additional time offset format...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/15147
  
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 #15147: [SPARK-17545] [SQL] Handle additional time offset...

2016-09-18 Thread nbeyer
GitHub user nbeyer opened a pull request:

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

[SPARK-17545] [SQL] Handle additional time offset formats of ISO 8601

## What changes were proposed in this pull request?
Allows flexibility in handling additional ISO 8601 time offset variants. 
The current parsing of ISO 8601 is exclusive to W3C's datetime note (and XML 
Schema datetime). This change will allow offset to be handled as "HH:MM", 
"HHMM" and "HH".

This is one suggested approach to handling these variants. The other 
suggestions are to switch back to SimpleDateFormat and utilize the 'X' pattern 
flag. Another suggestion is to wait for a future release of commons-lang. 
Either of these suggestions can be implemented later over top of this 
suggestion.

## How was this patch tested?
The patch was tested by running all existing unit tests and by augmenting 
the existing tests with additional assertions.




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

$ git pull https://github.com/nbeyer/spark master

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

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


commit db2e1f0c0b6bbf4683bbc2b876500141961becc0
Author: Nathan Beyer 
Date:   2016-09-19T02:23:05Z

Handle additional time offset formats of ISO 8601




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

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



[GitHub] spark issue #15146: [SPARK-17590][SQL] Analyze CTE definitions at once

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15146
  
**[Test build #65586 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65586/consoleFull)**
 for PR 15146 at commit 
[`394495d`](https://github.com/apache/spark/commit/394495d0effc0c217ff6bb0df92e664636e0dfa0).


---
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 #15146: [SPARK-17590][SQL] Analyze CTE definitions at onc...

2016-09-18 Thread viirya
GitHub user viirya opened a pull request:

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

[SPARK-17590][SQL] Analyze CTE definitions at once

## What changes were proposed in this pull request?

We substitute logical plan with CTE definitions in the analyzer rule 
CTESubstitution. A CTE definition can be used in the logical plan for multiple 
times, and its analyzed logical plan should be the same. We should not analyze 
CTE definitions multiple times when they are reused in the query.

## How was this patch tested?

Jenkins tests.




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

$ git pull https://github.com/viirya/spark-1 cte-analysis-once

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

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


commit 394495d0effc0c217ff6bb0df92e664636e0dfa0
Author: Liang-Chi Hsieh 
Date:   2016-09-19T02:13:19Z

Analyze CTE definition at once.




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

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



[GitHub] spark issue #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create externa...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create externa...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create externa...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

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



[GitHub] spark issue #14803: [SPARK-17153][SQL] Should read partition data when readi...

2016-09-18 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/14803
  
ping @marmbrus @zsxwing Would you mind to take a look this and provide your 
feedback? If this is not going to be fixed, please let me know too. This is a 
small change and I don't think it should be waiting for such long time. 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 #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create ...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15145#discussion_r79321615
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 ---
@@ -509,7 +509,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with TestHiv
   assert(
 intercept[AnalysisException] {
   sparkSession.catalog.createExternalTable("createdJsonTable", 
jsonFilePath.toString)
-}.getMessage.contains("Table default.createdJsonTable already 
exists."),
+}.getMessage.contains("Table `default`.`createdJsonTable` 
already exists."),
--- End diff --

Actually, I have a question about `unquotedString`. When should we issue an 
`unquotedString`? Unless we explicitly specify it, [we always use 
`quotedString`](https://github.com/apache/spark/blob/f036dd7ce727b40877337da66d687214786c4f14/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala#L40)
 for `TableIdentifier`. For `FunctionIdentifier`, [we use an 
`unquotedString`](https://github.com/apache/spark/blob/f036dd7ce727b40877337da66d687214786c4f14/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/identifiers.scala#L75).

Now, very few places are issuing `unquotedString` for table identifiers. 
Should correct them to make all the messages consistent?


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

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



[GitHub] spark issue #14452: [SPARK-16849][SQL][WIP] Improve subquery execution by de...

2016-09-18 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/14452
  
@hvanhovell @davies I rethink this PR in recent days. The changes includes 
some hacky change and are too big to review. I would like to separate it to 
individual small PRs which can be reviewed separately.




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

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



[GitHub] spark issue #14597: [SPARK-17017][MLLIB][ML] add a chiSquare Selector based ...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14597
  
**[Test build #65585 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65585/consoleFull)**
 for PR 14597 at commit 
[`88d2143`](https://github.com/apache/spark/commit/88d2143989a4219020100999c53bb1186fce5d1c).


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

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



[GitHub] spark issue #13680: [SPARK-15962][SQL] Introduce implementation with a dense...

2016-09-18 Thread kiszk
Github user kiszk commented on the issue:

https://github.com/apache/spark/pull/13680
  
@cloud-fan would it be possible to review this? I think that I implemented 
your suggestions.


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

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



[GitHub] spark issue #14995: [Test Only][SPARK-6235][CORE]Address various 2G limits

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14995
  
**[Test build #65584 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65584/consoleFull)**
 for PR 14995 at commit 
[`57833e3`](https://github.com/apache/spark/commit/57833e3a2b315294a1a8384d3fcee3926d4c55ce).


---
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 #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create ...

2016-09-18 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15145#discussion_r79320510
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 ---
@@ -509,7 +509,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with TestHiv
   assert(
 intercept[AnalysisException] {
   sparkSession.catalog.createExternalTable("createdJsonTable", 
jsonFilePath.toString)
-}.getMessage.contains("Table default.createdJsonTable already 
exists."),
+}.getMessage.contains("Table `default`.`createdJsonTable` 
already exists."),
--- End diff --

we should use `unquotedString` there


---
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 #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create ...

2016-09-18 Thread cloud-fan
Github user cloud-fan commented on a diff in the pull request:

https://github.com/apache/spark/pull/15145#discussion_r79320458
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/MetastoreDataSourcesSuite.scala
 ---
@@ -509,7 +509,7 @@ class MetastoreDataSourcesSuite extends QueryTest with 
SQLTestUtils with TestHiv
   assert(
 intercept[AnalysisException] {
   sparkSession.catalog.createExternalTable("createdJsonTable", 
jsonFilePath.toString)
-}.getMessage.contains("Table default.createdJsonTable already 
exists."),
+}.getMessage.contains("Table `default`.`createdJsonTable` 
already exists."),
--- End diff --

I think we may generate wrong error message, can you take a look at 
`CreateDatasourceCommand`?


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15053
  
Hi @mortada , I am sorry that I am a bit being noisy here but I just took a 
look for myself.

I resembled the PySpark structure and made a draft for myself.

```python
"""
>>> print df
[1, 2, 3]
>>> print df2
[]
"""

import doctest
import sys


def my_function(a, b):
"""
>>> my_function(df[0], df[1])
2
"""
return a * b


if __name__ == "__main__":
globs = globals()
globs['df'] = [1, 2, 3]
globs['df2'] = []
doctest.testmod(sys.modules[__name__], globs=globs)
```

If we do this, we would be able to just add 

```
"""
>>> print df
[1, 2, 3]
>>> print df2
[]
"""
```

on the top as a docstring and remove all the duplicated dataframe 
definitions in each doc string.

I guess this will be a minimal change and also satisfy your demands here 
with making the docstring clear.


---
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 #15054: [SPARK-17502] [SQL] Fix Multiple Bugs in DDL Stat...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/15054#discussion_r79320015
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/command/tables.scala ---
@@ -65,7 +64,11 @@ case class CreateTableLikeCommand(
 s"Source table in CREATE TABLE LIKE does not exist: 
'$sourceTable'")
 }
 
-val sourceTableDesc = catalog.getTableMetadata(sourceTable)
+val sourceTableDesc = if (catalog.isTemporaryTable(sourceTable)) {
+  catalog.getTempViewMetadata(sourceTable.table)
--- End diff --

Yeah, you are right. Let me fix it. 


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

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



[GitHub] spark issue #15136: [SPARK-17581] [SQL] Invalidate Statistics After Some ALT...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15136
  
Will do more investigation on this.


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

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



[GitHub] spark issue #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create externa...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15145
  
**[Test build #65583 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65583/consoleFull)**
 for PR 15145 at commit 
[`a7713de`](https://github.com/apache/spark/commit/a7713de3300c02aaba758a8e75364f8a7c8cbc41).


---
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 #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create ...

2016-09-18 Thread gatorsmile
GitHub user gatorsmile opened a pull request:

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

[SPARK-17589] [TEST] [2.0] Fix test case `create external table` in 
MetastoreDataSourcesSuite

### What changes were proposed in this pull request?
This PR is to fix a test failure on the branch 2.0 builds:

https://amplab.cs.berkeley.edu/jenkins/view/Spark%20QA%20Test%20(Dashboard)/job/spark-branch-2.0-test-maven-hadoop-2.7/711/
```
Error Message

"Table `default`.`createdJsonTable` already exists.;" did not contain 
"Table default.createdJsonTable already exists." We should complain that 
createdJsonTable already exists
```

### How was this patch tested?
N/A

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

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

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

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


commit a7713de3300c02aaba758a8e75364f8a7c8cbc41
Author: gatorsmile 
Date:   2016-09-19T00:44:46Z

fix.




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

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



[GitHub] spark issue #15145: [SPARK-17589] [TEST] [2.0] Fix test case `create externa...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15145
  
cc @cloud-fan @srowen 


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

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



[GitHub] spark issue #15122: [SPARK-17569] Make StructuredStreaming FileStreamSource ...

2016-09-18 Thread yhuai
Github user yhuai commented on the issue:

https://github.com/apache/spark/pull/15122
  
@petermaxlee I believe you will get a runtime exception saying that the 
file does not exist. 

Also, regarding your options 2, are you suggesting that users of structured 
streaming to use such a mock fs? Or you are suggesting that structured 
streaming to use such a fs. Also, why LocalFS is related to this case? 


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

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



[GitHub] spark issue #15142: [SPARK-17297] [DOCS] Clarify window/slide duration as ab...

2016-09-18 Thread peteb4ker
Github user peteb4ker commented on the issue:

https://github.com/apache/spark/pull/15142
  
Looks great, thx Sean. 


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

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



[GitHub] spark issue #15138: [SPARK-17583][SQL] Remove uesless rowSeparator variable ...

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15138
  
Yes, you are right and also yes, the purpose of the setting is to prevent 
OOM 
([documentation](https://github.com/uniVocity/univocity-parsers/blob/1ebd7e290b3f3d71a7ac9b8c5e3d1cfc220c8c96/src/main/java/com/univocity/parsers/common/CommonSettings.java#L35-L37)).
 I believe this limit was initially set by @falaki and I remember I have a 
positive answer when I try to increase this value.

If this is a normal case, it'd make sense if we set explicit limit because 
it is possible to try to read a whole file as a value within a column. However, 
I guess we are already reading and parsing line by line via `LineRecordReader` 
and via [CsvReader.parseLine(line: 
String)](https://github.com/apache/spark/blob/511f52f8423e151b0d0133baf040d34a0af3d422/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVParser.scala#L59).
 Therefore, I think the limit can't exceed the length of each line which I 
think is okay as a default value.

BTW, Apache common CSV does not have this limit IIRU.


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

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



[GitHub] spark issue #15099: [SPARK-17541][SQL] fix some DDL bugs about table managem...

2016-09-18 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/15099
  
Let me do a quick fix. 


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

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



[GitHub] spark issue #15114: [SPARK-17473][SQL] fixing docker integration tests error...

2016-09-18 Thread JoshRosen
Github user JoshRosen commented on the issue:

https://github.com/apache/spark/pull/15114
  
I tried running this in SBT and ran into a bunch of spurious exceptions 
from logging code:

```
SLF4J: Failed toString() invocation on an object of type 
[com.spotify.docker.client.messages.ProgressMessage]
Reported exception:
java.lang.NoClassDefFoundError: com/google/common/base/MoreObjects
at 
com.spotify.docker.client.messages.ProgressMessage.toString(ProgressMessage.java:136)
at 
org.slf4j.helpers.MessageFormatter.safeObjectAppend(MessageFormatter.java:300)
at 
org.slf4j.helpers.MessageFormatter.deeplyAppendParameter(MessageFormatter.java:272)
at 
org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:234)
at 
org.slf4j.helpers.MessageFormatter.arrayFormat(MessageFormatter.java:173)
at 
org.slf4j.helpers.MessageFormatter.format(MessageFormatter.java:151)
at 
org.slf4j.impl.Log4jLoggerAdapter.info(Log4jLoggerAdapter.java:346)
at 
com.spotify.docker.client.LoggingPullHandler.progress(LoggingPullHandler.java:45)
at 
com.spotify.docker.client.ProgressStream.tail(ProgressStream.java:74)
at 
com.spotify.docker.client.DefaultDockerClient.pull(DefaultDockerClient.java:905)
at 
com.spotify.docker.client.DefaultDockerClient.pull(DefaultDockerClient.java:879)
at 
com.spotify.docker.client.DefaultDockerClient.pull(DefaultDockerClient.java:873)
at 
org.apache.spark.sql.jdbc.DockerJDBCIntegrationSuite.beforeAll(DockerJDBCIntegrationSuite.scala:98)
at 
org.scalatest.BeforeAndAfterAll$class.beforeAll(BeforeAndAfterAll.scala:187)
at 
org.apache.spark.SparkFunSuite.beforeAll(SparkFunSuite.scala:29)
at 
org.scalatest.BeforeAndAfterAll$class.run(BeforeAndAfterAll.scala:253)
at org.apache.spark.SparkFunSuite.run(SparkFunSuite.scala:29)
at 
org.scalatest.tools.Framework.org$scalatest$tools$Framework$$runSuite(Framework.scala:357)
at 
org.scalatest.tools.Framework$ScalaTestTask.execute(Framework.scala:502)
at sbt.ForkMain$Run$2.call(ForkMain.java:296)
at sbt.ForkMain$Run$2.call(ForkMain.java:286)
at java.util.concurrent.FutureTask.run(FutureTask.java:266)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
at java.lang.Thread.run(Thread.java:745)
```

This didn't lead to test failures but made the test output a bit noisy 
while it was downloading images for the first time. This seems like it might be 
tricky to fix, though, so I'm going to merge this now to unblock developers who 
might rely on these tests. Thanks!


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15053
  
I haven't checked if package level docstring can define global level 
variables to be accessed from other docstrings. So, I would like to defer this 
to @holdenk (if you are not sure too, then we all together can look into this 
deeper).

I think it'd better if we follow @holdenk's comment first as I blieve this 
requires minimal change first and let's see if we all agree with fixing it in 
that way. If not, we could follow my comment as a safe choice.


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread mortada
Github user mortada commented on the issue:

https://github.com/apache/spark/pull/15053
  
@HyukjinKwon thanks for your help! I'm happy to complete this PR and follow 
what you suggest for testing. 

How would the package level docstring work? The goal (which I think we all 
agree on) is to be able to let the user easily see how the input is set up for 
each function in the docstring in a self-contained way. 


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

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



[GitHub] spark issue #15053: [Doc] improve python API docstrings

2016-09-18 Thread HyukjinKwon
Github user HyukjinKwon commented on the issue:

https://github.com/apache/spark/pull/15053
  
Hi @mortada, do you mind if I ask to address mine or @holdenk' comment? If 
you find any problem with testing, I am willing to take over this which I will 
ask comitters to credit this to 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 issue #15143: [SPARK-17584][Test] - Add unit test coverage for TaskSta...

2016-09-18 Thread erenavsarogullari
Github user erenavsarogullari commented on the issue:

https://github.com/apache/spark/pull/15143
  
Hi @rxin, 

Firstly, thanks for quick reply.

I was thinking for unit-test coverage perspective and a starter point to 
contribute project but it is ok for me if PR is closed 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 #15127: [SPARK-17571][SQL] AssertOnQuery.condition should...

2016-09-18 Thread asfgit
Github user asfgit closed the pull request at:

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


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

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



[GitHub] spark issue #15127: [SPARK-17571][SQL] AssertOnQuery.condition should always...

2016-09-18 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/15127
  
Merging in master/2.0.



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

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



[GitHub] spark issue #15143: [SPARK-17584][Test] - Add unit test coverage for TaskSta...

2016-09-18 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/15143
  
hm I agree having good unit test coverage is important -- this seems too 
trivial 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 issue #15142: [SPARK-17297] [DOCS] Clarify window/slide duration as ab...

2016-09-18 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/15142
  
LGTM


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

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



[GitHub] spark issue #3062: [SPARK-1406] Mllib pmml model export

2016-09-18 Thread manugarri
Github user manugarri commented on the issue:

https://github.com/apache/spark/pull/3062
  
im not sure if this is the right place to ask, but is there any plan to 
implement PMML export from pyspark? Cant find anything on the pyspark docs.


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

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



[GitHub] spark issue #15144: [SPARK-17587][PYTHON][MLLIB] SparseVector __getitem__ sh...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15144: [SPARK-17587][PYTHON][MLLIB] SparseVector __getitem__ sh...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15144: [SPARK-17587][PYTHON][MLLIB] SparseVector __getitem__ sh...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15144
  
**[Test build #65582 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65582/consoleFull)**
 for PR 15144 at commit 
[`4b90ee5`](https://github.com/apache/spark/commit/4b90ee591a4e914b8b156927e718d0e088ddce05).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #15134: [SPARK-17580][CORE]Add random UUID as app name while app...

2016-09-18 Thread sadikovi
Github user sadikovi commented on the issue:

https://github.com/apache/spark/pull/15134
  
@phalodi Does this solve (intend to solve) situation when spark-submit is 
launched with empty app name? Currently, as of 1.6, it will use empty 
application name.


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

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



[GitHub] spark issue #15142: [SPARK-17297] [DOCS] Clarify window/slide duration as ab...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15142: [SPARK-17297] [DOCS] Clarify window/slide duration as ab...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #15142: [SPARK-17297] [DOCS] Clarify window/slide duration as ab...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15142
  
**[Test build #65581 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65581/consoleFull)**
 for PR 15142 at commit 
[`61a0141`](https://github.com/apache/spark/commit/61a014158d689033f262ade75a1a3fe83d7b4ea9).
 * This patch passes all tests.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #15144: [SPARK-17587][PYTHON][MLLIB] SparseVector __getitem__ sh...

2016-09-18 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/15144
  
**[Test build #65582 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/65582/consoleFull)**
 for PR 15144 at commit 
[`4b90ee5`](https://github.com/apache/spark/commit/4b90ee591a4e914b8b156927e718d0e088ddce05).


---
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 #15144: [SPARK-17587][PYTHON][MLLIB] SparseVector __getit...

2016-09-18 Thread zero323
GitHub user zero323 opened a pull request:

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

[SPARK-17587][PYTHON][MLLIB] SparseVector __getitem__ should follow 
__getitem__ contract

## What changes were proposed in this pull request?

Replaces ValueError with IndexError when index passed to `ml` / `mllib` 
`SparseVector.__getitem__` is out of range. This ensures correct iteration 
behavior.

## How was this patch tested?

PySpark `ml` / `mllib` unit tests. Additional unit tests to prove that the 
problem has been resolved.



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

$ git pull https://github.com/zero323/spark SPARK-17587

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

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


commit 250560811497be5f3de5f6165350940dbf8e54c1
Author: zero323 
Date:   2016-09-18T19:15:14Z

Replace ValueError with IndexError in mllib.linalg

commit 13d39400072d37f3b9ae9596b59ef4d6d579234b
Author: zero323 
Date:   2016-09-18T19:19:07Z

Replace ValueError with IndexError in ml.linalg

commit aa10114f0ee515485d3f0d0ed9430fbd19349974
Author: zero323 
Date:   2016-09-18T19:58:14Z

Add tests for ml.linalg.SparseVector iteration

commit 4b90ee591a4e914b8b156927e718d0e088ddce05
Author: zero323 
Date:   2016-09-18T20:00:13Z

Add tests for mllib.linalg.SparseVector iteration




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

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



[GitHub] spark issue #15143: [SPARK-17584][Test] - Add unit test coverage for TaskSta...

2016-09-18 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #14444: [SPARK-16839] [SQL] redundant aliases after cleanupAlias...

2016-09-18 Thread eyalfa
Github user eyalfa commented on the issue:

https://github.com/apache/spark/pull/1
  
@cloud-fan, please see this 
[https://github.com/apache/spark/blob/1dbb725dbef30bf7633584ce8efdb573f2d92bca/sql/core/src/main/scala/org/apache/spark/sql/functions.scala#L1104-L1115](url),
 it seems that there's a strong guarantee that CreateStruct respects aliases 
and other types of named columns. furthermore, CreateNamedStruct is missing 
from this api/dsl which leaves users with no other viable way to name their 
structs attributes.

btw, can you please retest this? thanks.


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

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



[GitHub] spark pull request #15143: [SPARK-17584][Test] - Add unit test coverage for ...

2016-09-18 Thread erenavsarogullari
GitHub user erenavsarogullari opened a pull request:

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

[SPARK-17584][Test] - Add unit test coverage for TaskState and ExecutorState

## What changes were proposed in this pull request?
- TaskState and ExecutorState expose isFailed and isFinished functions. It 
can be useful to add test coverage for different states. Currently, Other enums 
do not expose any functions so this PR aims just these two enums.
- `private` access modifier is added for Finished Task States Set
- A minor doc change is added.

## How was this patch tested?
New Unit tests are added and run locally.

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

$ git pull https://github.com/erenavsarogullari/spark SPARK-17584

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

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


commit 02c3b1b3a529f4954f55c8856a1e5d0dc218e6cf
Author: erenavsarogullari 
Date:   2016-09-18T16:12:08Z

New UT Cases are added for TaskState and ExecutorState enums.




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

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



[GitHub] spark issue #15114: [SPARK-17473][SQL] fixing docker integration tests error...

2016-09-18 Thread lresende
Github user lresende commented on the issue:

https://github.com/apache/spark/pull/15114
  
I verified this works on native docker in linux with :
build/mvn -Pdocker-integration-tests -Pscala-2.11 -pl 
:spark-docker-integration-tests_2.11 clean compile test

LGTM.




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

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



[GitHub] spark issue #14762: [SPARK-16962][CORE][SQL] Fix misaligned record accesses ...

2016-09-18 Thread sumansomasundar
Github user sumansomasundar commented on the issue:

https://github.com/apache/spark/pull/14762
  
@srowen I ran dev/lint-java, removed few additional white spaces, and 
shortened few lines longer than 100 characters, then rebased it. 


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

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



[GitHub] spark issue #14444: [SPARK-16839] [SQL] redundant aliases after cleanupAlias...

2016-09-18 Thread eyalfa
Github user eyalfa commented on the issue:

https://github.com/apache/spark/pull/1
  
@hvanhovell , I'm currently trying your approach of testing `ne.resolved` 
prior to accessing `ne.name`.
tests are running as I write here, but a quick dive into the 
`NamedExpression` hierarchy revealed some more cases (other than `Star`) where 
the names are not available prior to analysis, i.e.`UnresolvedAlias` and 
`MultiAlias`, there's also `UnresolvedAttribute` which isn't resolved but its 
name is always available (doesn't throw).
even though I've tried I couldn't come up with a use case that combines 
these expressions with `CreateStruct`, I'd appreciate if any of you guys can 
shed some light on these classes, especially `UnresolvedAlias`. I'm a bit 
worried about this one as I suspect `CreateStruct.apply` might encounter one, 
looking at `Analyzer` shows that `ResolveAliases` is executed after 
`ResolveFunctions` so I guess my concern is at least theoretically possible. 


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

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



[GitHub] spark issue #14981: [SPARK-17418] Remove Kinesis artifacts from Spark releas...

2016-09-18 Thread lresende
Github user lresende commented on the issue:

https://github.com/apache/spark/pull/14981
  
@srowen Please don't get me wrong, I don't have any interest on this 
extension either, but just want to make sure we start doing the right thing for 
Apache Spark. I will try to ping some of the long time Apache members on the 
legal-discussion and see if we can get a definitive/authoritative answer for 
this particular case.


---
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   >