[GitHub] spark pull request #16750: [SPARK-18937][SQL] Timezone support in CSV/JSON p...

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

https://github.com/apache/spark/pull/16750#discussion_r99531337
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/json/JSONOptions.scala
 ---
@@ -31,10 +31,11 @@ import 
org.apache.spark.sql.catalyst.util.{CaseInsensitiveMap, CompressionCodecs
  * Most of these map directly to Jackson's internal options, specified in 
[[JsonParser.Feature]].
  */
 private[sql] class JSONOptions(
-@transient private val parameters: CaseInsensitiveMap)
+@transient private val parameters: CaseInsensitiveMap, 
defaultTimeZoneId: String)
--- End diff --

shouldn't the `timeZoneId` just an option in `parameters` with key 
`timeZoneId`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16680: [SPARK-16101][SQL] Refactoring CSV schema inferen...

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

https://github.com/apache/spark/pull/16680#discussion_r99530531
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVUtils.scala
 ---
@@ -0,0 +1,134 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources.csv
+
+import org.apache.spark.sql.Dataset
+import org.apache.spark.sql.functions._
+import org.apache.spark.sql.types._
+
+object CSVUtils {
+  /**
+   * Filter ignorable rows for CSV dataset (lines empty and starting with 
`comment`).
+   * This is currently being used in CSV schema inference.
+   */
+  def filterCommentAndEmpty(lines: Dataset[String], options: CSVOptions): 
Dataset[String] = {
+// Note that this was separately made by SPARK-18362. Logically, this 
should be the same
+// with the one below, `filterCommentAndEmpty` but execution path is 
different. One of them
+// might have to be removed in the near future if possible.
+import lines.sqlContext.implicits._
+val nonEmptyLines = lines.filter(length(trim($"value")) > 0)
+if (options.isCommentSet) {
+  nonEmptyLines.filter(!$"value".startsWith(options.comment.toString))
+} else {
+  nonEmptyLines
+}
+  }
+
+  /**
+   * Filter ignorable rows for CSV iterator (lines empty and starting with 
`comment`).
+   * This is currently being used in CSV reading path and CSV schema 
inference.
+   */
+  def filterCommentAndEmpty(iter: Iterator[String], options: CSVOptions): 
Iterator[String] = {
+iter.filter { line =>
+  line.trim.nonEmpty && !line.startsWith(options.comment.toString)
+}
+  }
+
+  /**
+   * Skip the given first line so that only data can remain in a dataset.
+   * This is similar with `dropHeaderLine` below and currently being used 
in CSV schema inference.
+   */
+  def filterHeaderLine(
+   iter: Iterator[String],
+   firstLine: String,
+   options: CSVOptions): Iterator[String] = {
+// Note that unlike actual CSV reading path, it simply filters the 
given first line. Therefore,
+// this skips the line same with the header if exists. One of them 
might have to be removed
+// in the near future if possible.
+if (options.headerFlag) {
+  iter.filterNot(_ == firstLine)
+} else {
+  iter
+}
+  }
+
+  /**
+   * Drop header line so that only data can remain.
+   * This is similar with `filterHeaderLine` above and currently being 
used in CSV reading path.
+   */
+  def dropHeaderLine(iter: Iterator[String], options: CSVOptions): 
Iterator[String] = {
+val nonEmptyLines = if (options.isCommentSet) {
+  val commentPrefix = options.comment.toString
+  iter.dropWhile { line =>
+line.trim.isEmpty || line.trim.startsWith(commentPrefix)
+  }
+} else {
+  iter.dropWhile(_.trim.isEmpty)
+}
+
+if (nonEmptyLines.hasNext) nonEmptyLines.drop(1)
+iter
+  }
+
+  /**
+   * Helper method that converts string representation of a character to 
actual character.
+   * It handles some Java escaped strings and throws exception if given 
string is longer than one
+   * character.
+   */
+  @throws[IllegalArgumentException]
+  def toChar(str: String): Char = {
+if (str.charAt(0) == '\\') {
+  str.charAt(1)
+  match {
+case 't' => '\t'
+case 'r' => '\r'
+case 'b' => '\b'
+case 'f' => '\f'
+case '\"' => '\"' // In case user changes quote char and uses \" 
as delimiter in options
+case '\'' => '\''
+case 'u' if str == """\u""" => '\u'
+case _ =>
+  throw new IllegalArgumentException(s"Unsupported special 
character for delimiter: $str")
+  }
+} else if (str.length == 1) {
+  str.charAt(0)
 

[GitHub] spark pull request #16680: [SPARK-16101][SQL] Refactoring CSV schema inferen...

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

https://github.com/apache/spark/pull/16680#discussion_r99530062
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVInferSchema.scala
 ---
@@ -39,22 +37,76 @@ private[csv] object CSVInferSchema {
* 3. Replace any null types with string type
*/
   def infer(
-  tokenRdd: RDD[Array[String]],
-  header: Array[String],
+  csv: Dataset[String],
--- End diff --

nit: `csvLines`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16680: [SPARK-16101][SQL] Refactoring CSV schema inferen...

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

https://github.com/apache/spark/pull/16680#discussion_r99530031
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/csv/CSVFileFormat.scala
 ---
@@ -170,32 +111,21 @@ class CSVFileFormat extends TextBasedFileFormat with 
DataSourceRegister {
 }
   }
 
-  // Consumes the header in the iterator.
-  CSVRelation.dropHeaderLine(file, lineIterator, csvOptions)
-
-  val filteredIter = lineIterator.filter { line =>
-line.trim.nonEmpty && !line.startsWith(commentPrefix)
+  val linesWithoutHeader = if (csvOptions.headerFlag && file.start == 
0) {
+// Note that if there are only comments in the first block, the 
header would probably
+// be not dropped.
+CSVUtils.dropHeaderLine(lines, csvOptions)
+  } else {
+lines
   }
 
+  val filteredLines = 
CSVUtils.filterCommentAndEmpty(linesWithoutHeader, csvOptions)
   val parser = new UnivocityParser(dataSchema, requiredSchema, 
csvOptions)
-  filteredIter.flatMap(parser.parse)
-}
-  }
-
-  /**
-   * Returns the first line of the first non-empty file in path
-   */
-  private def findFirstLine(options: CSVOptions, lines: Dataset[String]): 
String = {
-import lines.sqlContext.implicits._
-val nonEmptyLines = lines.filter(length(trim($"value")) > 0)
-if (options.isCommentSet) {
-  
nonEmptyLines.filter(!$"value".startsWith(options.comment.toString)).first()
-} else {
-  nonEmptyLines.first()
+  filteredLines.flatMap(parser.parse)
 }
   }
 
-  private def readText(
--- End diff --

seems `readAsLines` is better?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16787
  
**[Test build #72426 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72426/testReport)**
 for PR 16787 at commit 
[`5ef2139`](https://github.com/apache/spark/commit/5ef2139a7628ea5d6568f56b3a87ad9b3cf1caed).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16803: [SPARK-19458][SQL]load hive jars from local repo which h...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16803
  
**[Test build #72425 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72425/testReport)**
 for PR 16803 at commit 
[`1bb31e5`](https://github.com/apache/spark/commit/1bb31e51a73565a07dc703edf51578762a47f5b2).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16787
  
**[Test build #72424 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72424/testReport)**
 for PR 16787 at commit 
[`352856b`](https://github.com/apache/spark/commit/352856bca3dff862f09253ec5a06cb55fff3e76a).
 * This patch **fails to build**.
 * 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 #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark issue #16738: [SPARK-19398] Change one misleading log in TaskSetManage...

2017-02-05 Thread jinxing64
Github user jinxing64 commented on the issue:

https://github.com/apache/spark/pull/16738
  
@kayousterhout 
Thanks a lot again : )


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

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



[GitHub] spark issue #16791: [SPARK-19409][SPARK-17213] Cleanup Parquet workarounds/h...

2017-02-05 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/16791
  
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 pull request #16803: [SPARK-19458][SQL]load hive jars from local repo ...

2017-02-05 Thread windpiger
Github user windpiger commented on a diff in the pull request:

https://github.com/apache/spark/pull/16803#discussion_r99528722
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -480,7 +479,12 @@ object SparkSubmit extends CommandLineUtils {
 sysProp = "spark.driver.cores"),
   OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER,
 sysProp = "spark.driver.supervise"),
-  OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = 
"spark.jars.ivy")
+  OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
+sysProp = "spark.jars.ivy"),
+  OptionAssigner(args.repositories, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
+sysProp = "spark.jars.repositories"),
--- End diff --

yes, it is used to store user's repos , then we can use it in download hive 
jars.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16757: [SPARK-18609][SPARK-18841][SQL] Fix redundant Ali...

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

https://github.com/apache/spark/pull/16757#discussion_r99528500
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -154,56 +155,97 @@ class SimpleTestOptimizer extends Optimizer(
   new SimpleCatalystConf(caseSensitiveAnalysis = true))
 
 /**
- * Removes the Project only conducting Alias of its child node.
- * It is created mainly for removing extra Project added in 
EliminateSerialization rule,
- * but can also benefit other operators.
+ * Remove redundant aliases from a query plan. A redundant alias is an 
alias that does not change
+ * the name or metadata of a column, and does not deduplicate it.
  */
-object RemoveAliasOnlyProject extends Rule[LogicalPlan] {
+object RemoveRedundantAliases extends Rule[LogicalPlan] {
+
   /**
-   * Returns true if the project list is semantically same as child 
output, after strip alias on
-   * attribute.
+   * Replace the attributes in an expression using the given mapping.
*/
-  private def isAliasOnly(
-  projectList: Seq[NamedExpression],
-  childOutput: Seq[Attribute]): Boolean = {
-if (projectList.length != childOutput.length) {
-  false
-} else {
-  stripAliasOnAttribute(projectList).zip(childOutput).forall {
-case (a: Attribute, o) if a semanticEquals o => true
-case _ => false
-  }
+  private def createAttributeMapping(current: LogicalPlan, next: 
LogicalPlan)
+  : Seq[(Attribute, Attribute)] = {
+current.output.zip(next.output).filterNot {
+  case (a1, a2) => a1.semanticEquals(a2)
 }
   }
 
-  private def stripAliasOnAttribute(projectList: Seq[NamedExpression]) = {
-projectList.map {
-  // Alias with metadata can not be stripped, or the metadata will be 
lost.
-  // If the alias name is different from attribute name, we can't 
strip it either, or we may
-  // accidentally change the output schema name of the root plan.
-  case a @ Alias(attr: Attribute, name) if a.metadata == 
Metadata.empty && name == attr.name =>
-attr
-  case other => other
-}
+  /**
+   * Remove the top-level alias from an expression when it is redundant.
+   */
+  private def removeRedundantAlias(e: Expression, blacklist: 
AttributeSet): Expression = e match {
+// Alias with metadata can not be stripped, or the metadata will be 
lost.
+// If the alias name is different from attribute name, we can't strip 
it either, or we
+// may accidentally change the output schema name of the root plan.
+case a @ Alias(attr: Attribute, name)
+  if a.metadata == Metadata.empty && name == attr.name && 
!blacklist.contains(attr) =>
+  attr
+case a => a
   }
 
-  def apply(plan: LogicalPlan): LogicalPlan = {
-val aliasOnlyProject = plan.collectFirst {
-  case p @ Project(pList, child) if isAliasOnly(pList, child.output) 
=> p
-}
+  /**
+   * Remove redundant alias expression from a LogicalPlan and its subtree. 
A blacklist is used to
+   * prevent the removal of seemingly redundant aliases used to 
deduplicate the input for a (self)
+   * join.
+   */
+  private def removeRedundantAliases(plan: LogicalPlan, blacklist: 
AttributeSet): LogicalPlan = {
+plan match {
+  // A join has to be treated differently, because the left and the 
right side of the join are
+  // not allowed to use the same attributes. We use a blacklist to 
prevent us from creating a
+  // situation in which this happens; the rule will only remove an 
alias if its child
+  // attribute is not on the black list.
+  case Join(left, right, joinType, condition) =>
+val newLeft = removeRedundantAliases(left, blacklist ++ 
right.outputSet)
+val newRight = removeRedundantAliases(right, blacklist ++ 
newLeft.outputSet)
+val mapping = AttributeMap(
+  createAttributeMapping(left, newLeft) ++
+  createAttributeMapping(right, newRight))
+val newCondition = condition.map(_.transform {
+  case a: Attribute => mapping.getOrElse(a, a)
+})
+Join(newLeft, newRight, joinType, newCondition)
+
+  case _ =>
+// Remove redundant aliases in the subtree(s).
+val currentNextAttrPairs = mutable.Buffer.empty[(Attribute, 
Attribute)]
+val newNode = plan.mapChildren { child =>
+  val newChild = removeRedundantAliases(child, blacklist)
+  currentNextAttrPairs ++= createAttributeMapping(child, newChild)
+  newChild
+}
 
-aliasOnlyProject.map { case proj =>
-  val 

[GitHub] spark pull request #16757: [SPARK-18609][SPARK-18841][SQL] Fix redundant Ali...

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

https://github.com/apache/spark/pull/16757#discussion_r99528351
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/Optimizer.scala
 ---
@@ -154,56 +155,97 @@ class SimpleTestOptimizer extends Optimizer(
   new SimpleCatalystConf(caseSensitiveAnalysis = true))
 
 /**
- * Removes the Project only conducting Alias of its child node.
- * It is created mainly for removing extra Project added in 
EliminateSerialization rule,
- * but can also benefit other operators.
+ * Remove redundant aliases from a query plan. A redundant alias is an 
alias that does not change
+ * the name or metadata of a column, and does not deduplicate it.
  */
-object RemoveAliasOnlyProject extends Rule[LogicalPlan] {
+object RemoveRedundantAliases extends Rule[LogicalPlan] {
+
   /**
-   * Returns true if the project list is semantically same as child 
output, after strip alias on
-   * attribute.
+   * Replace the attributes in an expression using the given mapping.
*/
-  private def isAliasOnly(
-  projectList: Seq[NamedExpression],
-  childOutput: Seq[Attribute]): Boolean = {
-if (projectList.length != childOutput.length) {
-  false
-} else {
-  stripAliasOnAttribute(projectList).zip(childOutput).forall {
-case (a: Attribute, o) if a semanticEquals o => true
-case _ => false
-  }
+  private def createAttributeMapping(current: LogicalPlan, next: 
LogicalPlan)
+  : Seq[(Attribute, Attribute)] = {
+current.output.zip(next.output).filterNot {
+  case (a1, a2) => a1.semanticEquals(a2)
 }
   }
 
-  private def stripAliasOnAttribute(projectList: Seq[NamedExpression]) = {
-projectList.map {
-  // Alias with metadata can not be stripped, or the metadata will be 
lost.
-  // If the alias name is different from attribute name, we can't 
strip it either, or we may
-  // accidentally change the output schema name of the root plan.
-  case a @ Alias(attr: Attribute, name) if a.metadata == 
Metadata.empty && name == attr.name =>
-attr
-  case other => other
-}
+  /**
+   * Remove the top-level alias from an expression when it is redundant.
+   */
+  private def removeRedundantAlias(e: Expression, blacklist: 
AttributeSet): Expression = e match {
+// Alias with metadata can not be stripped, or the metadata will be 
lost.
+// If the alias name is different from attribute name, we can't strip 
it either, or we
+// may accidentally change the output schema name of the root plan.
+case a @ Alias(attr: Attribute, name)
+  if a.metadata == Metadata.empty && name == attr.name && 
!blacklist.contains(attr) =>
+  attr
+case a => a
   }
 
-  def apply(plan: LogicalPlan): LogicalPlan = {
-val aliasOnlyProject = plan.collectFirst {
-  case p @ Project(pList, child) if isAliasOnly(pList, child.output) 
=> p
-}
+  /**
+   * Remove redundant alias expression from a LogicalPlan and its subtree. 
A blacklist is used to
+   * prevent the removal of seemingly redundant aliases used to 
deduplicate the input for a (self)
+   * join.
+   */
+  private def removeRedundantAliases(plan: LogicalPlan, blacklist: 
AttributeSet): LogicalPlan = {
+plan match {
+  // A join has to be treated differently, because the left and the 
right side of the join are
+  // not allowed to use the same attributes. We use a blacklist to 
prevent us from creating a
+  // situation in which this happens; the rule will only remove an 
alias if its child
+  // attribute is not on the black list.
+  case Join(left, right, joinType, condition) =>
+val newLeft = removeRedundantAliases(left, blacklist ++ 
right.outputSet)
+val newRight = removeRedundantAliases(right, blacklist ++ 
newLeft.outputSet)
+val mapping = AttributeMap(
+  createAttributeMapping(left, newLeft) ++
+  createAttributeMapping(right, newRight))
+val newCondition = condition.map(_.transform {
+  case a: Attribute => mapping.getOrElse(a, a)
+})
+Join(newLeft, newRight, joinType, newCondition)
+
+  case _ =>
+// Remove redundant aliases in the subtree(s).
+val currentNextAttrPairs = mutable.Buffer.empty[(Attribute, 
Attribute)]
+val newNode = plan.mapChildren { child =>
+  val newChild = removeRedundantAliases(child, blacklist)
+  currentNextAttrPairs ++= createAttributeMapping(child, newChild)
+  newChild
+}
 
-aliasOnlyProject.map { case proj =>
-  val 

[GitHub] spark issue #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16787
  
**[Test build #72424 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72424/testReport)**
 for PR 16787 at commit 
[`352856b`](https://github.com/apache/spark/commit/352856bca3dff862f09253ec5a06cb55fff3e76a).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16138: [SPARK-16609] Add to_date/to_timestamp with format funct...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72422/
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 #16138: [SPARK-16609] Add to_date/to_timestamp with format funct...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16138
  
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 #16138: [SPARK-16609] Add to_date/to_timestamp with format funct...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16138
  
**[Test build #72422 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72422/testReport)**
 for PR 16138 at commit 
[`c3063a2`](https://github.com/apache/spark/commit/c3063a2178f6daea3063f63b90edd3404d3048eb).
 * 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 #16138: [SPARK-16609] Add to_date/to_timestamp with forma...

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

https://github.com/apache/spark/pull/16138#discussion_r99527096
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/DateFunctionsSuite.scala ---
@@ -354,31 +354,58 @@ class DateFunctionsSuite extends QueryTest with 
SharedSQLContext {
   test("function to_date") {
 val d1 = Date.valueOf("2015-07-22")
 val d2 = Date.valueOf("2015-07-01")
+val d3 = Date.valueOf("2014-12-31")
 val t1 = Timestamp.valueOf("2015-07-22 10:00:00")
 val t2 = Timestamp.valueOf("2014-12-31 23:59:59")
+val t3 = Timestamp.valueOf("2014-12-31 23:59:59")
 val s1 = "2015-07-22 10:00:00"
 val s2 = "2014-12-31"
-val df = Seq((d1, t1, s1), (d2, t2, s2)).toDF("d", "t", "s")
+val s3 = "2014-31-12"
+val df = Seq((d1, t1, s1), (d2, t2, s2), (d3, t3, s3)).toDF("d", "t", 
"s")
 
 checkAnswer(
   df.select(to_date(col("t"))),
-  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2014-12-31"
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")),
+Row(Date.valueOf("2014-12-31"
 checkAnswer(
   df.select(to_date(col("d"))),
-  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2015-07-01"
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")),
+Row(Date.valueOf("2014-12-31"
 checkAnswer(
   df.select(to_date(col("s"))),
-  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2014-12-31"
+  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2014-12-31")), Row(null)))
 
 checkAnswer(
   df.selectExpr("to_date(t)"),
-  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2014-12-31"
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")),
+Row(Date.valueOf("2014-12-31"
 checkAnswer(
   df.selectExpr("to_date(d)"),
-  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2015-07-01"
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")),
+Row(Date.valueOf("2014-12-31"
 checkAnswer(
   df.selectExpr("to_date(s)"),
-  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2014-12-31"
+  Seq(Row(Date.valueOf("2015-07-22")), 
Row(Date.valueOf("2014-12-31")), Row(null)))
+
+  // Now with format
+checkAnswer(
+  df.select(to_date(col("t"), "-MM-dd")),
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")),
+Row(Date.valueOf("2014-12-31"
+checkAnswer(
+  df.select(to_date(col("d"), "-MM-dd")),
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2015-07-01")),
+Row(Date.valueOf("2014-12-31"
+checkAnswer(
+  df.select(to_date(col("s"), "-MM-dd")),
+  Seq(Row(Date.valueOf("2015-07-22")), Row(Date.valueOf("2014-12-31")),
+Row(Date.valueOf("2016-07-12"
+
+//  now switch format
+checkAnswer(
+  df.select(to_date(col("s"), "-dd-MM")),
+  Seq(Row(Date.valueOf("2016-10-07")), Row(Date.valueOf("2016-07-12")),
--- End diff --

I'm surprised, converting `2015-07-22 10:00:00` to date with format 
`-dd-MM` will result to `2016-10-07`? When we convert `2014-31-12` to date 
with default format, it returns null.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16787
  
**[Test build #72423 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72423/testReport)**
 for PR 16787 at commit 
[`489cb16`](https://github.com/apache/spark/commit/489cb16c0d09440dc4b46dec649f8848b9b31c87).
 * This patch **fails to build**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class ResolveSQLOnFile(sparkSession: SparkSession) extends 
Rule[LogicalPlan] `
  * `class ResolveHiveSerdeTable(session: SparkSession) extends 
Rule[LogicalPlan] `


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16738: [SPARK-19398] Change one misleading log in TaskSetManage...

2017-02-05 Thread kayousterhout
Github user kayousterhout commented on the issue:

https://github.com/apache/spark/pull/16738
  
LGTM! Thanks for fixing this!  I'll merge this tomorrow once tests pass and 
pending any further comments from @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 #16789: Fix for SPARK-19444

2017-02-05 Thread anshbansal
Github user anshbansal commented on the issue:

https://github.com/apache/spark/pull/16789
  
@srowen Will read them and get back


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16738: [SPARK-19398] Change one misleading log in TaskSetManage...

2017-02-05 Thread jinxing64
Github user jinxing64 commented on the issue:

https://github.com/apache/spark/pull/16738
  
@kayousterhout 
Thanks a lot for helping this pr thus far. I think the proposal is quite 
clear. I've already refined. Please take another look.


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

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



[GitHub] spark issue #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16787
  
**[Test build #72423 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72423/testReport)**
 for PR 16787 at commit 
[`489cb16`](https://github.com/apache/spark/commit/489cb16c0d09440dc4b46dec649f8848b9b31c87).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16815: [SPARK-19407][SS] defaultFS is used FileSystem.get inste...

2017-02-05 Thread uncleGen
Github user uncleGen commented on the issue:

https://github.com/apache/spark/pull/16815
  
cc @srowen @zsxwing 


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

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



[GitHub] spark issue #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener callback...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16664
  
Just finished this round of reviews. Thanks!

This PR enables the QueryExecutionListener when users using the 
DataFrameWriter methods. However, it still misses the other code paths, 
especially, the DDL statements. For example, CTAS when using the `sql()` API. 
cc @sameeragarwal @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 #16762: [SPARK-19419] [SPARK-19420] Fix the cross join detection

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16762
  
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 #16762: [SPARK-19419] [SPARK-19420] Fix the cross join detection

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16762
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72421/
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 #16762: [SPARK-19419] [SPARK-19420] Fix the cross join detection

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16762
  
**[Test build #72421 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72421/testReport)**
 for PR 16762 at commit 
[`d02baee`](https://github.com/apache/spark/commit/d02baee13bd91b4ad3aace8946d19e08abad7a24).
 * 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener callback...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16664
  
Could you update the PR title to `[SPARK-18120][SQL] Call 
QueryExecutionListener callback methods for DataFrameWriter methods`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16816: Code style improvement

2017-02-05 Thread zhoucen
Github user zhoucen commented on the issue:

https://github.com/apache/spark/pull/16816
  
@uncleGen Ok!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16816: Code style improvement

2017-02-05 Thread uncleGen
Github user uncleGen commented on the issue:

https://github.com/apache/spark/pull/16816
  
It is too tiny to make a pr, so let us close it, OK?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16738: [SPARK-19398] Change one misleading log in TaskSe...

2017-02-05 Thread kayousterhout
Github user kayousterhout commented on a diff in the pull request:

https://github.com/apache/spark/pull/16738#discussion_r99523053
  
--- Diff: 
core/src/main/scala/org/apache/spark/scheduler/TaskSetManager.scala ---
@@ -799,7 +799,7 @@ private[spark] class TaskSetManager(
 if (successful(index)) {
   logInfo(
 s"Task ${info.id} in stage ${taskSet.id} (TID $tid) failed, " +
-"but another instance of the task has already succeeded, " +
+"but it is already marked as successful, " +
--- End diff --

How about 

"Task ${info.id} in stage ${taskSet.id} (TID $tid) failed, but the task 
will not be re-executed (either because the task failed with a fetch failure, 
so the previous stage needs to be re-run, or because a different copy of the 
task has already succeeded)"


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16737: [SPARK-19397] [SQL] Make option names of LIBSVM a...

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

https://github.com/apache/spark/pull/16737#discussion_r99522959
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/source/libsvm/LibSVMOptions.scala ---
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.source.libsvm
+
+import org.apache.spark.sql.catalyst.util.CaseInsensitiveMap
+
+/**
+ * Options for the LibSVM data source.
+ */
+private[libsvm] class LibSVMOptions(@transient private val parameters: 
CaseInsensitiveMap)
+  extends Serializable {
+
+  import LibSVMOptions._
+
+  def this(parameters: Map[String, String]) = this(new 
CaseInsensitiveMap(parameters))
+
+  /**
+   * Number of features. If unspecified or nonpositive, the number of 
features will be determined
+   * automatically at the cost of one additional pass.
+   */
+  val numFeatures = parameters.get(NUMFEATURES).map(_.toInt)
+
+  val isSparse = parameters.getOrElse(VECTORTYPE, SPARSEVECTORTYPE) match {
+case SPARSEVECTORTYPE => true
+case DENSEVECTORTYPE => false
+case o => throw new IllegalArgumentException(s"Invalid value `$o` for 
parameter " +
+  s"`$VECTORTYPE`. Expected types are `sparse` and `dense`.")
+  }
+}
+
+private[libsvm] object LibSVMOptions {
+  val NUMFEATURES = "numFeatures"
--- End diff --

nit: `NUM_FEATURES`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16726: [SPARK-19390][SQL] Replace the unnecessary usages...

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

https://github.com/apache/spark/pull/16726#discussion_r99522843
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveTableScanSuite.scala
 ---
@@ -166,13 +166,11 @@ class HiveTableScanSuite extends HiveComparisonTest 
with SQLTestUtils with TestH
  |PARTITION (p1='a',p2='c',p3='c',p4='d',p5='e')
  |SELECT v.id
""".stripMargin)
-val plan = sql(
-  s"""
- |SELECT * FROM $table
-   """.stripMargin).queryExecution.sparkPlan
+val plan = sql(s"SELECT * FROM $table").queryExecution.sparkPlan
 val relation = plan.collectFirst {
   case p: HiveTableScanExec => p.relation
 }.get
+// This test case is to verify `hiveQlTable` and 
`getHiveQlPartitions()`
 val tableCols = relation.hiveQlTable.getCols
--- End diff --

can we remove `hiveQlTable` now?


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

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



[GitHub] spark issue #16816: Code style improvement

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16816
  
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 #16809: [SPARK-19463][SQL]refresh cache after the InsertIntoHado...

2017-02-05 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/16809
  
This is a behavior change rather than a bug fix, but I think this new 
behavior makes more sense. cc @gatorsmile to confirm.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16816: Code style improvement

2017-02-05 Thread zhoucen
GitHub user zhoucen opened a pull request:

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

Code style improvement

Use 2-space indentation in general.


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

$ git pull https://github.com/zhoucen/spark patch-1

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

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


commit 1d5a2590e976270e7b961bfc48d4908227a7319e
Author: 周岑 
Date:   2017-02-06T06:12:32Z

Code style improvement

Use 2-space indentation in general.




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99521880
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/SparkSQLQueryExecutionListenerSuite.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * 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
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkContext, SparkFunSuite}
+import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.util.{OutputParams, QueryExecutionListener}
+
+/**
+ * Test cases for the property 'spark.sql.queryExecutionListeners' that 
adds the
+ * @see `QueryExecutionListener` to a @see `SparkSession`
+ */
+class SparkSQLQueryExecutionListenerSuite
+extends SparkFunSuite
+with MockitoSugar
+with BeforeAndAfterEach {
+
+  override def afterEach(): Unit = {
+SparkSession.clearActiveSession()
+SparkSession.clearDefaultSession()
+SparkContext.clearActiveContext()
+  }
+
+  test("Creation of SparkContext with non-existent QueryExecutionListener 
class fails fast") {
+intercept[ClassNotFoundException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
"non.existent.QueryExecutionListener")
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("QueryExecutionListener that doesn't have a default constructor 
fails fast") {
+intercept[InstantiationException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
classOf[NoZeroArgConstructorListener].getName)
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("Normal QueryExecutionListeners gets added as listeners") {
+val sparkSession = SparkSession
+  .builder()
+  .master("local")
+  .config("mykey", "myvalue")
+  .config("spark.sql.queryExecutionListeners",
+classOf[NormalQueryExecutionListener].getName + " ,"
+  + classOf[AnotherQueryExecutionListener].getName)
+  .getOrCreate()
+assert(SparkSession.getDefaultSession.isDefined)
+assert(NormalQueryExecutionListener.successCount === 0)
+assert(NormalQueryExecutionListener.failureCount === 0)
+assert(AnotherQueryExecutionListener.successCount === 0)
+assert(AnotherQueryExecutionListener.failureCount === 0)
+sparkSession.listenerManager.onSuccess("test1", mock[QueryExecution], 
0)
+assert(NormalQueryExecutionListener.successCount === 1)
+assert(NormalQueryExecutionListener.failureCount === 0)
+assert(AnotherQueryExecutionListener.successCount === 1)
+assert(AnotherQueryExecutionListener.failureCount === 0)
+sparkSession.listenerManager.onFailure("test2", mock[QueryExecution], 
new Exception)
+assert(NormalQueryExecutionListener.successCount === 1)
+assert(NormalQueryExecutionListener.failureCount === 1)
+assert(AnotherQueryExecutionListener.successCount === 1)
+assert(AnotherQueryExecutionListener.failureCount === 1)
+  }
+}
+
+class NoZeroArgConstructorListener(myString: String) extends 
QueryExecutionListener {
+
+  override def onSuccess(
+  funcName: String,
+  qe: QueryExecution,
+  durationNs: Long,
+  options: Option[OutputParams]
+  ): Unit = {}
--- End diff --

Nit: -> `options: Option[OutputParams]): Unit = {}`


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


[GitHub] spark pull request #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99521699
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/SparkSQLQueryExecutionListenerSuite.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * 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
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkContext, SparkFunSuite}
+import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.util.{OutputParams, QueryExecutionListener}
+
+/**
+ * Test cases for the property 'spark.sql.queryExecutionListeners' that 
adds the
+ * @see `QueryExecutionListener` to a @see `SparkSession`
+ */
+class SparkSQLQueryExecutionListenerSuite
+extends SparkFunSuite
+with MockitoSugar
+with BeforeAndAfterEach {
+
+  override def afterEach(): Unit = {
+SparkSession.clearActiveSession()
+SparkSession.clearDefaultSession()
+SparkContext.clearActiveContext()
+  }
+
+  test("Creation of SparkContext with non-existent QueryExecutionListener 
class fails fast") {
+intercept[ClassNotFoundException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
"non.existent.QueryExecutionListener")
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
--- End diff --

The same here. `isEmpty`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99521679
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/SparkSQLQueryExecutionListenerSuite.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * 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
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkContext, SparkFunSuite}
+import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.util.{OutputParams, QueryExecutionListener}
+
+/**
+ * Test cases for the property 'spark.sql.queryExecutionListeners' that 
adds the
+ * @see `QueryExecutionListener` to a @see `SparkSession`
+ */
+class SparkSQLQueryExecutionListenerSuite
+extends SparkFunSuite
+with MockitoSugar
+with BeforeAndAfterEach {
+
+  override def afterEach(): Unit = {
+SparkSession.clearActiveSession()
+SparkSession.clearDefaultSession()
+SparkContext.clearActiveContext()
+  }
+
+  test("Creation of SparkContext with non-existent QueryExecutionListener 
class fails fast") {
+intercept[ClassNotFoundException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
"non.existent.QueryExecutionListener")
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("QueryExecutionListener that doesn't have a default constructor 
fails fast") {
+intercept[InstantiationException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
classOf[NoZeroArgConstructorListener].getName)
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
--- End diff --

`assert(SparkSession.getDefaultSession.isEmpty)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99521620
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/SparkSQLQueryExecutionListenerSuite.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * 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
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkContext, SparkFunSuite}
+import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.util.{OutputParams, QueryExecutionListener}
+
+/**
+ * Test cases for the property 'spark.sql.queryExecutionListeners' that 
adds the
+ * @see `QueryExecutionListener` to a @see `SparkSession`
+ */
+class SparkSQLQueryExecutionListenerSuite
+extends SparkFunSuite
+with MockitoSugar
+with BeforeAndAfterEach {
+
+  override def afterEach(): Unit = {
+SparkSession.clearActiveSession()
+SparkSession.clearDefaultSession()
+SparkContext.clearActiveContext()
+  }
+
+  test("Creation of SparkContext with non-existent QueryExecutionListener 
class fails fast") {
+intercept[ClassNotFoundException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
"non.existent.QueryExecutionListener")
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("QueryExecutionListener that doesn't have a default constructor 
fails fast") {
+intercept[InstantiationException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
classOf[NoZeroArgConstructorListener].getName)
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("Normal QueryExecutionListeners gets added as listeners") {
+val sparkSession = SparkSession
+  .builder()
+  .master("local")
+  .config("mykey", "myvalue")
+  .config("spark.sql.queryExecutionListeners",
+classOf[NormalQueryExecutionListener].getName + " ,"
+  + classOf[AnotherQueryExecutionListener].getName)
+  .getOrCreate()
+assert(SparkSession.getDefaultSession.isDefined)
+assert(NormalQueryExecutionListener.successCount === 0)
+assert(NormalQueryExecutionListener.failureCount === 0)
+assert(AnotherQueryExecutionListener.successCount === 0)
+assert(AnotherQueryExecutionListener.failureCount === 0)
+sparkSession.listenerManager.onSuccess("test1", mock[QueryExecution], 
0)
+assert(NormalQueryExecutionListener.successCount === 1)
+assert(NormalQueryExecutionListener.failureCount === 0)
+assert(AnotherQueryExecutionListener.successCount === 1)
+assert(AnotherQueryExecutionListener.failureCount === 0)
+sparkSession.listenerManager.onFailure("test2", mock[QueryExecution], 
new Exception)
+assert(NormalQueryExecutionListener.successCount === 1)
+assert(NormalQueryExecutionListener.failureCount === 1)
+assert(AnotherQueryExecutionListener.successCount === 1)
+assert(AnotherQueryExecutionListener.failureCount === 1)
+  }
+}
+
+class NoZeroArgConstructorListener(myString: String) extends 
QueryExecutionListener {
+
+  override def onSuccess(
+  funcName: String,
+  qe: QueryExecution,
+  durationNs: Long,
+  options: Option[OutputParams]
+  ): Unit = {}
+
+  override def onFailure(
+  funcName: String,
+  qe: QueryExecution,
+  exception: Exception,
+  options: Option[OutputParams]
+  ): Unit = {}
+}
+
+class NormalQueryExecutionListener extends QueryExecutionListener {
+
+  override def onSuccess(
+  funcName: String,
+  qe: QueryExecution,
+  durationNs: Long,
+  

[GitHub] spark pull request #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99521628
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/SparkSQLQueryExecutionListenerSuite.scala
 ---
@@ -0,0 +1,149 @@
+/*
+ * 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
+
+import org.scalatest.BeforeAndAfterEach
+import org.scalatest.mock.MockitoSugar
+
+import org.apache.spark.{SparkContext, SparkFunSuite}
+import org.apache.spark.sql.execution.QueryExecution
+import org.apache.spark.sql.util.{OutputParams, QueryExecutionListener}
+
+/**
+ * Test cases for the property 'spark.sql.queryExecutionListeners' that 
adds the
+ * @see `QueryExecutionListener` to a @see `SparkSession`
+ */
+class SparkSQLQueryExecutionListenerSuite
+extends SparkFunSuite
+with MockitoSugar
+with BeforeAndAfterEach {
+
+  override def afterEach(): Unit = {
+SparkSession.clearActiveSession()
+SparkSession.clearDefaultSession()
+SparkContext.clearActiveContext()
+  }
+
+  test("Creation of SparkContext with non-existent QueryExecutionListener 
class fails fast") {
+intercept[ClassNotFoundException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
"non.existent.QueryExecutionListener")
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("QueryExecutionListener that doesn't have a default constructor 
fails fast") {
+intercept[InstantiationException] {
+  SparkSession
+.builder()
+.master("local")
+.config("spark.sql.queryExecutionListeners", 
classOf[NoZeroArgConstructorListener].getName)
+.getOrCreate()
+}
+assert(!SparkSession.getDefaultSession.isDefined)
+  }
+
+  test("Normal QueryExecutionListeners gets added as listeners") {
+val sparkSession = SparkSession
+  .builder()
+  .master("local")
+  .config("mykey", "myvalue")
+  .config("spark.sql.queryExecutionListeners",
+classOf[NormalQueryExecutionListener].getName + " ,"
+  + classOf[AnotherQueryExecutionListener].getName)
+  .getOrCreate()
+assert(SparkSession.getDefaultSession.isDefined)
+assert(NormalQueryExecutionListener.successCount === 0)
+assert(NormalQueryExecutionListener.failureCount === 0)
+assert(AnotherQueryExecutionListener.successCount === 0)
+assert(AnotherQueryExecutionListener.failureCount === 0)
+sparkSession.listenerManager.onSuccess("test1", mock[QueryExecution], 
0)
+assert(NormalQueryExecutionListener.successCount === 1)
+assert(NormalQueryExecutionListener.failureCount === 0)
+assert(AnotherQueryExecutionListener.successCount === 1)
+assert(AnotherQueryExecutionListener.failureCount === 0)
+sparkSession.listenerManager.onFailure("test2", mock[QueryExecution], 
new Exception)
+assert(NormalQueryExecutionListener.successCount === 1)
+assert(NormalQueryExecutionListener.failureCount === 1)
+assert(AnotherQueryExecutionListener.successCount === 1)
+assert(AnotherQueryExecutionListener.failureCount === 1)
+  }
+}
+
+class NoZeroArgConstructorListener(myString: String) extends 
QueryExecutionListener {
+
+  override def onSuccess(
+  funcName: String,
+  qe: QueryExecution,
+  durationNs: Long,
+  options: Option[OutputParams]
+  ): Unit = {}
+
+  override def onFailure(
+  funcName: String,
+  qe: QueryExecution,
+  exception: Exception,
+  options: Option[OutputParams]
+  ): Unit = {}
+}
+
+class NormalQueryExecutionListener extends QueryExecutionListener {
+
+  override def onSuccess(
+  funcName: String,
+  qe: QueryExecution,
+  durationNs: Long,
+  

[GitHub] spark issue #16783: [SPARK-19441] [SQL] Remove IN type coercion from Promote...

2017-02-05 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/16783
  
+1


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

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



[GitHub] spark issue #16810: [SPARK-19464][CORE][YARN][test-hadoop2.6] Remove support...

2017-02-05 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/16810
  
It's great, +1.


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

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



[GitHub] spark pull request #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99520106
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -1047,4 +1047,14 @@ object StaticSQLConf {
   "SQL configuration and the current database.")
 .booleanConf
 .createWithDefault(false)
+
+  val QUERY_EXECUTION_LISTENERS = 
buildConf("spark.sql.queryExecutionListeners")
+.doc("A comma-separated list of classes that implement 
QueryExecutionListener. When creating " +
+  "a SparkSession, instances of these listeners will be added to it. 
These classes " +
+  "needs to have a zero-argument constructor. If the specified class 
can't be found or" +
+  " the class specified doesn't have a valid constructor the 
SparkSession creation " +
+  "will fail with an exception.")
+.stringConf
+.toSequence
+.createWithDefault(Nil)
--- End diff --

Not sure whether we should make it internal or external. Let the others 
decide it. Either is fine to me.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99520045
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala 
---
@@ -893,6 +896,12 @@ object SparkSession {
 }
   }
 
+  private def createQueryExecutionListeners(conf: SparkConf): 
Seq[QueryExecutionListener] = {
+conf.get(StaticSQLConf.QUERY_EXECUTION_LISTENERS)
+  .map(Utils.classForName(_))
+  .map(_.newInstance().asInstanceOf[QueryExecutionListener])
--- End diff --

Simply throwing `ClassNotFoundException` might not be good to end users, if 
we plan to make this SQL configuration external. 

Could you use the try and catch to issue a better error message when we are 
unable to create/initialize the class? 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 #16810: [SPARK-19464][CORE][YARN][test-hadoop2.6] Remove ...

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

https://github.com/apache/spark/pull/16810#discussion_r99519864
  
--- Diff: docs/building-spark.md ---
@@ -63,57 +63,30 @@ with Maven profile settings and so on like the direct 
Maven build. Example:
 
 This will build Spark distribution along with Python pip and R packages. 
For more information on usage, run `./dev/make-distribution.sh --help`
 
-## Specifying the Hadoop Version
-
-Because HDFS is not protocol-compatible across versions, if you want to 
read from HDFS, you'll need to build Spark against the specific HDFS version in 
your environment. You can do this through the `hadoop.version` property. If 
unset, Spark will build against Hadoop 2.2.0 by default. Note that certain 
build profiles are required for particular Hadoop versions:
-
-
-  
-Hadoop versionProfile required
-  
-  
-2.2.xhadoop-2.2
-2.3.xhadoop-2.3
-2.4.xhadoop-2.4
-2.6.xhadoop-2.6
-2.7.x and later 2.xhadoop-2.7
-  
-
-
-Note that support for versions of Hadoop before 2.6 are deprecated as of 
Spark 2.1.0 and may be
-removed in Spark 2.2.0.
+## Specifying the Hadoop Version and Enabling YARN
 
+You can specify the exact version of Hadoop to compile against through the 
`hadoop.version` property. 
+If unset, Spark will build against Hadoop 2.6.0 by default.
--- End diff --

`2.6.5 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 issue #16787: [SPARK-19448][SQL]optimize some duplication functions in...

2017-02-05 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/16787
  
please resolve the conflict, now you need to remove the duplicated 
functions between HiveUtils and HiveClientImpl


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16636: [SPARK-19279] [SQL] Infer Schema for Hive Serde T...

2017-02-05 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16636: [SPARK-19279] [SQL] Infer Schema for Hive Serde Tables a...

2017-02-05 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/16636
  
LGTM, merging to master!


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

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



[GitHub] spark pull request #16636: [SPARK-19279] [SQL] Infer Schema for Hive Serde T...

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

https://github.com/apache/spark/pull/16636#discussion_r99519100
  
--- Diff: 
sql/hive/src/test/scala/org/apache/spark/sql/hive/execution/HiveDDLSuite.scala 
---
@@ -79,6 +79,25 @@ class HiveDDLSuite
 }
   }
 
+  test("create a hive table without schema") {
+import testImplicits._
+withTempPath { tempDir =>
+  withTable("tab1", "tab2") {
+(("a", "b") :: Nil).toDF().write.json(tempDir.getCanonicalPath)
+
+var e = intercept[AnalysisException] { sql("CREATE TABLE tab1 
USING hive") }.getMessage
+assert(e.contains("Unable to infer the schema. The schema 
specification is required to " +
+  "create the table `default`.`tab1`"))
+
+e = intercept[AnalysisException] {
+  sql(s"CREATE TABLE tab2 location '${tempDir.getCanonicalPath}'")
--- End diff --

is there any hive serde that can infer schema from data files?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99518656
  
--- Diff: sql/core/src/main/scala/org/apache/spark/sql/SparkSession.scala 
---
@@ -893,6 +896,12 @@ object SparkSession {
 }
   }
 
+  private def createQueryExecutionListeners(conf: SparkConf): 
Seq[QueryExecutionListener] = {
+conf.get(StaticSQLConf.QUERY_EXECUTION_LISTENERS)
+  .map(Utils.classForName(_))
--- End diff --

Nit: -> `.map(Utils.classForName)`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99518378
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -1047,4 +1047,14 @@ object StaticSQLConf {
   "SQL configuration and the current database.")
 .booleanConf
 .createWithDefault(false)
+
+  val QUERY_EXECUTION_LISTENERS = 
buildConf("spark.sql.queryExecutionListeners")
+.doc("A comma-separated list of classes that implement 
QueryExecutionListener. When creating " +
+  "a SparkSession, instances of these listeners will be added to it. 
These classes " +
+  "needs to have a zero-argument constructor. If the specified class 
can't be found or" +
+  " the class specified doesn't have a valid constructor the 
SparkSession creation " +
--- End diff --

Nit: please move the starting space to the end of the last line.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99518295
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -1047,4 +1047,14 @@ object StaticSQLConf {
   "SQL configuration and the current database.")
 .booleanConf
 .createWithDefault(false)
+
+  val QUERY_EXECUTION_LISTENERS = 
buildConf("spark.sql.queryExecutionListeners")
+.doc("A comma-separated list of classes that implement 
QueryExecutionListener. When creating " +
+  "a SparkSession, instances of these listeners will be added to it. 
These classes " +
+  "needs to have a zero-argument constructor. If the specified class 
can't be found or" +
+  " the class specified doesn't have a valid constructor the 
SparkSession creation " +
--- End diff --

`the class specified` -> `the specified class`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99518239
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala ---
@@ -1047,4 +1047,14 @@ object StaticSQLConf {
   "SQL configuration and the current database.")
 .booleanConf
 .createWithDefault(false)
+
+  val QUERY_EXECUTION_LISTENERS = 
buildConf("spark.sql.queryExecutionListeners")
+.doc("A comma-separated list of classes that implement 
QueryExecutionListener. When creating " +
+  "a SparkSession, instances of these listeners will be added to it. 
These classes " +
+  "needs to have a zero-argument constructor. If the specified class 
can't be found or" +
--- End diff --

Nit: `needs` -> `need`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16803: [SPARK-19458][SQL]load hive jars from local repo ...

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

https://github.com/apache/spark/pull/16803#discussion_r99518229
  
--- Diff: 
core/src/test/scala/org/apache/spark/deploy/SparkSubmitUtilsSuite.scala ---
@@ -258,4 +259,53 @@ class SparkSubmitUtilsSuite extends SparkFunSuite with 
BeforeAndAfterAll {
   assert(jarPath.indexOf("mydep") >= 0, "should find dependency")
 }
   }
+
+  test("search for artifact taking order from user defined repositories to 
default repositories") {
+val main = new MavenCoordinate("a", "b", "0.1")
+
+def isSameFile(left: String, right: String): Boolean = {
+  val leftInput: FileInputStream = new FileInputStream(left)
+  val leftMd5 = UTF8String.fromString(org.apache.commons.codec
--- End diff --

why convert it to `UTF8String` and compare?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16803: [SPARK-19458][SQL]load hive jars from local repo ...

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

https://github.com/apache/spark/pull/16803#discussion_r99518186
  
--- Diff: core/src/main/scala/org/apache/spark/deploy/SparkSubmit.scala ---
@@ -480,7 +479,12 @@ object SparkSubmit extends CommandLineUtils {
 sysProp = "spark.driver.cores"),
   OptionAssigner(args.supervise.toString, STANDALONE | MESOS, CLUSTER,
 sysProp = "spark.driver.supervise"),
-  OptionAssigner(args.ivyRepoPath, STANDALONE, CLUSTER, sysProp = 
"spark.jars.ivy")
+  OptionAssigner(args.ivyRepoPath, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
+sysProp = "spark.jars.ivy"),
+  OptionAssigner(args.repositories, ALL_CLUSTER_MGRS, ALL_DEPLOY_MODES,
+sysProp = "spark.jars.repositories"),
--- End diff --

this is a new option?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99518130
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -218,7 +247,14 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   bucketSpec = getBucketSpec,
   options = extraOptions.toMap)
 
-dataSource.write(mode, df)
+val destination = source match {
+  case "jdbc" => extraOptions.get(JDBCOptions.JDBC_TABLE_NAME)
+  case _ => extraOptions.get("path")
--- End diff --

For the external data source connectors, it might not have `path`. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16803: [SPARK-19458][SQL]load hive jars from local repo which h...

2017-02-05 Thread cloud-fan
Github user cloud-fan commented on the issue:

https://github.com/apache/spark/pull/16803
  
This should be taged as `[BUILD]`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener c...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on a diff in the pull request:

https://github.com/apache/spark/pull/16664#discussion_r99518083
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/DataFrameWriter.scala ---
@@ -218,7 +247,14 @@ final class DataFrameWriter[T] private[sql](ds: 
Dataset[T]) {
   bucketSpec = getBucketSpec,
   options = extraOptions.toMap)
 
-dataSource.write(mode, df)
+val destination = source match {
+  case "jdbc" => extraOptions.get(JDBCOptions.JDBC_TABLE_NAME)
--- End diff --

For JDBC, the source value might not be `jdbc`. For example, `jDbC`, 
`JDBC`, `org.apache.spark.sql.jdbc.DefaultSource`, `org.apache.spark.sql.jdbc`


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16138: [SPARK-16609] Add to_date/to_timestamp with format funct...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16138: [SPARK-16609] Add to_date/to_timestamp with format funct...

2017-02-05 Thread anabranch
Github user anabranch commented on the issue:

https://github.com/apache/spark/pull/16138
  
@felixcheung thanks, made those changes :). Hopefully this will start 
passing sometime :P


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16809: [SPARK-19463][SQL]refresh cache after the InsertIntoHado...

2017-02-05 Thread windpiger
Github user windpiger commented on the issue:

https://github.com/apache/spark/pull/16809
  
cc @cloud-fan @gatorsmile 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16762: [SPARK-19419] [SPARK-19420] Fix the cross join detection

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16760: [SPARK-18872][SQL][TESTS] New test cases for EXISTS subq...

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on the issue:

https://github.com/apache/spark/pull/16760
  
@dilipbiswal Does that mean your test cases cover all the EXISTS scenarios 
in `SubquerySuite.scala`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16762: [SPARK-19419] [SPARK-19420] Fix the cross join detection

2017-02-05 Thread gatorsmile
Github user gatorsmile commented on the issue:

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


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

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



[GitHub] spark issue #16815: [SPARK-19407][SS] defaultFS is used FileSystem.get inste...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16815
  
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 #16815: [SPARK-19407][SS] defaultFS is used FileSystem.get inste...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16815
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72420/
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 #16815: [SPARK-19407][SS] defaultFS is used FileSystem.get inste...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16815
  
**[Test build #72420 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72420/testReport)**
 for PR 16815 at commit 
[`c8c3e4b`](https://github.com/apache/spark/commit/c8c3e4b55bd453bad687fcffe9d3ee4b7a728c84).
 * 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 #16809: [SPARK-19463][SQL]refresh cache after the InsertIntoHado...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16809
  
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 #16809: [SPARK-19463][SQL]refresh cache after the InsertIntoHado...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16809
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72419/
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 #16809: [SPARK-19463][SQL]refresh cache after the InsertIntoHado...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16809
  
**[Test build #72419 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72419/testReport)**
 for PR 16809 at commit 
[`15350f1`](https://github.com/apache/spark/commit/15350f1df5951eacaa68afdae24e5a4cd1948b65).
 * 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 #16727: [SPARK-19421][ML][PySpark] Remove numClasses and ...

2017-02-05 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16727: [SPARK-19421][ML][PySpark] Remove numClasses and numFeat...

2017-02-05 Thread holdenk
Github user holdenk commented on the issue:

https://github.com/apache/spark/pull/16727
  
This looks good, merging into master. Thanks for doing this cleanup work 
@zhengruifeng :)


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16797: [SPARK-19455][SQL] Add option for case-insensitive Parqu...

2017-02-05 Thread ericl
Github user ericl commented on the issue:

https://github.com/apache/spark/pull/16797
  
I agree that bringing back schema inference would be cleaner. One problem 
with doing something parquet-specific is that this would need to be repeated 
with each file format, e.g. orc, csv, json, unless we made case sensitivity an 
API option as noted above.

Per @viirya 's suggestion, it seems ideal to only do schema inference for 
older tables, and provide some command to update the table schema to preserve 
the case. This way we don't regress performance on 2.1 datasets or have to 
worry about schemas changing during optimization.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16785: [SPARK-19443][SQL][WIP] The function to generate constra...

2017-02-05 Thread viirya
Github user viirya commented on the issue:

https://github.com/apache/spark/pull/16785
  
The rewritten logic is not correct. I am working to improve this with other 
approach.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16727: [SPARK-19421][ML][PySpark] Remove numClasses and numFeat...

2017-02-05 Thread zhengruifeng
Github user zhengruifeng commented on the issue:

https://github.com/apache/spark/pull/16727
  
ping @yanboliang ?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16620: [SPARK-19263] DAGScheduler should avoid sending conflict...

2017-02-05 Thread jinxing64
Github user jinxing64 commented on the issue:

https://github.com/apache/spark/pull/16620
  
@squito 
Thanks a lot for helping this PR thus far.
I've added unit test in `DAGSchedulerSuite`, but not sure if it is exactly 
what you suggest. 
I created a `mockTaskSchedulerImpl`. Since lots of status are maintained in 
`TaskScheudlerImpl`, I have to trigger the event by `resourceOffers`, 
`handleSuccessfulTask`, `handleFailedTask`.
Please give another look at this when you have time. Really appreciate if 
you could help.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16614: [SPARK-19260] Spaces or "%20" in path parameter are not ...

2017-02-05 Thread zuotingbing
Github user zuotingbing commented on the issue:

https://github.com/apache/spark/pull/16614
  
@srowen Happy Chinese New Year !
Sorry for the late reply. It sounds more sensible to test this behavior 
everywhere since all the test cases used the directory, right? Thank you for 
your patience and guidance. 


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16815: [SPARK-19407][SS] defaultFS is used FileSystem.get inste...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16815: [SPARK-19407][SS] defaultFS is used FileSystem.ge...

2017-02-05 Thread uncleGen
GitHub user uncleGen opened a pull request:

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

[SPARK-19407][SS] defaultFS is used FileSystem.get instead of getting it 
from uri scheme

## What changes were proposed in this pull request?

```
Caused by: java.lang.IllegalArgumentException: Wrong FS: 
s3a://**/checkpoint/7b2231a3-d845-4740-bfa3-681850e5987f/metadata, 
expected: file:///
at org.apache.hadoop.fs.FileSystem.checkPath(FileSystem.java:649)
at 
org.apache.hadoop.fs.RawLocalFileSystem.pathToFile(RawLocalFileSystem.java:82)
at 
org.apache.hadoop.fs.RawLocalFileSystem.deprecatedGetFileStatus(RawLocalFileSystem.java:606)
at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileLinkStatusInternal(RawLocalFileSystem.java:824)
at 
org.apache.hadoop.fs.RawLocalFileSystem.getFileStatus(RawLocalFileSystem.java:601)
at 
org.apache.hadoop.fs.FilterFileSystem.getFileStatus(FilterFileSystem.java:421)
at org.apache.hadoop.fs.FileSystem.exists(FileSystem.java:1426)
at 
org.apache.spark.sql.execution.streaming.StreamMetadata$.read(StreamMetadata.scala:51)
at 
org.apache.spark.sql.execution.streaming.StreamExecution.(StreamExecution.scala:100)
at 
org.apache.spark.sql.streaming.StreamingQueryManager.createQuery(StreamingQueryManager.scala:232)
at 
org.apache.spark.sql.streaming.StreamingQueryManager.startQuery(StreamingQueryManager.scala:269)
at 
org.apache.spark.sql.streaming.DataStreamWriter.start(DataStreamWriter.scala:262)
```

Can easily replicate on spark standalone cluster by providing checkpoint 
location uri scheme anything other than "file://" and not overriding in config.

WorkAround  --conf spark.hadoop.fs.defaultFS=s3a://somebucket or set it in 
sparkConf or spark-default.conf

## How was this patch tested?

existing ut

cc @srowen @zsxwing 


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

$ git pull https://github.com/uncleGen/spark SPARK-19407

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

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


commit c8c3e4b55bd453bad687fcffe9d3ee4b7a728c84
Author: uncleGen 
Date:   2017-02-06T01:36:29Z

defaultFS is used FileSystem.get instead of getting it from uri scheme




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16809: [SPARK-19463][SQL]refresh cache after the InsertIntoHado...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16809
  
**[Test build #72419 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72419/testReport)**
 for PR 16809 at commit 
[`15350f1`](https://github.com/apache/spark/commit/15350f1df5951eacaa68afdae24e5a4cd1948b65).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16638: [SPARK-19115] [SQL] Supporting Create External Table Lik...

2017-02-05 Thread ouyangxiaochen
Github user ouyangxiaochen commented on the issue:

https://github.com/apache/spark/pull/16638
  
Happy Chinese New Year ! @gatorsmile 
The Spring Festival holiday just ended, and I return to work today, what 
work do I need to do?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16776: [SPARK-19436][SQL] Add missing tests for approxQuantile

2017-02-05 Thread zhengruifeng
Github user zhengruifeng commented on the issue:

https://github.com/apache/spark/pull/16776
  
@gatorsmile Updated! Thanks for reviewing!


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener callback...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16664
  
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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener callback...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16664
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72417/
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 #16664: [SPARK-18120 ][SQL] Call QueryExecutionListener callback...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16664
  
**[Test build #72417 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72417/testReport)**
 for PR 16664 at commit 
[`a0c7c22`](https://github.com/apache/spark/commit/a0c7c22e4097adbdf12e52db37c26a2246d4eddd).
 * 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 #16814: [SPARK-19467][ML][PYTHON]Remove cyclic imports from pysp...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16814
  
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 #16814: [SPARK-19467][ML][PYTHON]Remove cyclic imports from pysp...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16814
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72418/
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 #16814: [SPARK-19467][ML][PYTHON]Remove cyclic imports from pysp...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16814
  
**[Test build #72418 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72418/testReport)**
 for PR 16814 at commit 
[`4c31ec7`](https://github.com/apache/spark/commit/4c31ec7bed204ec3b420a971c12dfdf1617c3b65).
 * 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 #16607: [SPARK-19247][ML] Save large word2vec models

2017-02-05 Thread asfgit
Github user asfgit closed the pull request at:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16607: [SPARK-19247][ML] Save large word2vec models

2017-02-05 Thread jkbradley
Github user jkbradley commented on the issue:

https://github.com/apache/spark/pull/16607
  
LGTM
Merging with master
Thanks @Krimit !


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16814: [SPARK-19467][ML][PYTHON]Remove cyclic imports from pysp...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16814
  
**[Test build #72418 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72418/testReport)**
 for PR 16814 at commit 
[`4c31ec7`](https://github.com/apache/spark/commit/4c31ec7bed204ec3b420a971c12dfdf1617c3b65).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16814: [SPARK-19467][ML][PYTHON]Remove cyclic imports fr...

2017-02-05 Thread zero323
GitHub user zero323 opened a pull request:

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

[SPARK-19467][ML][PYTHON]Remove cyclic imports from pyspark.ml.pipeline

## What changes were proposed in this pull request?

Remove cyclic imports between `pyspark.ml.pipeline` and `pyspark.ml`.

## How was this patch tested?

Existing unit tests.

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

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

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

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


commit 4c31ec7bed204ec3b420a971c12dfdf1617c3b65
Author: zero323 
Date:   2017-02-05T23:43:46Z

Remove cyclic imports from pyspark.ml.pipeline




---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #16798: [SPARK-18873][SQL][TEST] New test cases for scalar subqu...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16798
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/72416/
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 #16798: [SPARK-18873][SQL][TEST] New test cases for scalar subqu...

2017-02-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/16798
  
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 #16798: [SPARK-18873][SQL][TEST] New test cases for scalar subqu...

2017-02-05 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/16798
  
**[Test build #72416 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/72416/testReport)**
 for PR 16798 at commit 
[`044d6a4`](https://github.com/apache/spark/commit/044d6a4927f77bb50631d74830a7a999a6d6fd26).
 * 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 #16810: [SPARK-19464][CORE][YARN][test-hadoop2.6] Remove support...

2017-02-05 Thread felixcheung
Github user felixcheung commented on the issue:

https://github.com/apache/spark/pull/16810
  
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 pull request #16810: [SPARK-19464][CORE][YARN][test-hadoop2.6] Remove ...

2017-02-05 Thread felixcheung
Github user felixcheung commented on a diff in the pull request:

https://github.com/apache/spark/pull/16810#discussion_r99503085
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/client/IsolatedClientLoader.scala
 ---
@@ -63,17 +63,16 @@ private[hive] object IsolatedClientLoader extends 
Logging {
 } catch {
   case e: RuntimeException if e.getMessage.contains("hadoop") =>
 // If the error message contains hadoop, it is probably 
because the hadoop
-// version cannot be resolved (e.g. it is a vendor specific 
version like
-// 2.0.0-cdh4.1.1). If it is the case, we will try just
-// "org.apache.hadoop:hadoop-client:2.4.0". 
"org.apache.hadoop:hadoop-client:2.4.0"
+// version cannot be resolved. If it is the case, we will try 
just
+// "org.apache.hadoop:hadoop-client:2.6.0". 
"org.apache.hadoop:hadoop-client:2.6.0"
 // is used just because we used to hard code it as the hadoop 
artifact to download.
-logWarning(s"Failed to resolve Hadoop artifacts for the 
version ${hadoopVersion}. " +
-  s"We will change the hadoop version from ${hadoopVersion} to 
2.4.0 and try again. " +
+logWarning(s"Failed to resolve Hadoop artifacts for the 
version $hadoopVersion. " +
+  s"We will change the hadoop version from $hadoopVersion to 
2.6.0 and try again. " +
   "Hadoop classes will not be shared between Spark and Hive 
metastore client. " +
   "It is recommended to set jars used by Hive metastore client 
through " +
   "spark.sql.hive.metastore.jars in the production 
environment.")
 sharesHadoopClasses = false
-(downloadVersion(resolvedVersion, "2.4.0", ivyPath), "2.4.0")
+(downloadVersion(resolvedVersion, "2.6.0", ivyPath), "2.6.0")
--- End diff --

should this be 2.6.5, matching `2.6.5`?


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file 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   >