[GitHub] spark pull request #13494: [SPARK-15752] [SQL] Optimize metadata only query ...

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/13494#discussion_r70316064
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuery.scala
 ---
@@ -0,0 +1,153 @@
+/*
+ * 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
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.catalog.{CatalogRelation, 
SessionCatalog}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.plans.logical._
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{HadoopFsRelation, 
LogicalRelation}
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * This rule optimizes the execution of queries that can be answered by 
looking only at
+ * partition-level metadata. This applies when all the columns scanned are 
partition columns, and
+ * the query has an aggregate operator that satisfies the following 
conditions:
+ * 1. aggregate expression is partition columns.
+ *  e.g. SELECT col FROM tbl GROUP BY col.
+ * 2. aggregate function on partition columns with DISTINCT.
+ *  e.g. SELECT col1, count(DISTINCT col2) FROM tbl GROUP BY col1.
+ * 3. aggregate function on partition columns which have same result w or 
w/o DISTINCT keyword.
+ *  e.g. SELECT col1, Max(col2) FROM tbl GROUP BY col1.
+ */
+case class OptimizeMetadataOnlyQuery(
+catalog: SessionCatalog,
+conf: SQLConf) extends Rule[LogicalPlan] {
+
+  def apply(plan: LogicalPlan): LogicalPlan = {
+if (!conf.optimizerMetadataOnly) {
+  return plan
+}
+
+plan.transform {
+  case a @ Aggregate(_, aggExprs, child @ 
PartitionedRelation(partAttrs, relation)) =>
+// We only apply this optimization when only partitioned 
attributes are scanned.
+if (a.references.subsetOf(partAttrs)) {
+  val aggFunctions = aggExprs.flatMap(_.collect {
+case agg: AggregateExpression => agg
+  })
+  val isAllDistinctAgg = aggFunctions.forall { agg =>
+agg.isDistinct || (agg.aggregateFunction match {
+  // `Max`, `Min`, `First` and `Last` are always distinct 
aggregate functions no matter
+  // they have DISTINCT keyword or not, as the result will be 
same.
+  case _: Max => true
+  case _: Min => true
+  case _: First => true
+  case _: Last => true
+  case _ => false
+})
+  }
+  if (isAllDistinctAgg) {
+
a.withNewChildren(Seq(replaceTableScanWithPartitionMetadata(child, relation)))
+  } else {
+a
+  }
+} else {
+  a
+}
+}
+  }
+
+  /**
+   * Transform the given plan, find its table scan nodes that matches the 
given relation, and then
+   * replace the table scan node with its corresponding partition values.
+   */
+  private def replaceTableScanWithPartitionMetadata(
+  child: LogicalPlan,
+  relation: LogicalPlan): LogicalPlan = {
+child transform {
+  case plan if plan eq relation =>
+relation match {
+  case l @ LogicalRelation(fsRelation: HadoopFsRelation, _, _) =>
+val partAttrs = PartitionedRelation.getPartitionAttrs(
+  fsRelation.partitionSchema.map(_.name), l)
+val partitionData = fsRelation.location.listFiles(filters = 
Nil)
+LocalRelation(partAttrs, partitionData.map(_.values))
+
+  case relation: CatalogRelation =>
+val partAttrs = PartitionedRelation.getPartitionAttrs(
+  relation.catalogTable.partitionColumnNames, relation)
+val partitionData = 

[GitHub] spark pull request #13494: [SPARK-15752] [SQL] Optimize metadata only query ...

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/13494#discussion_r70316221
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/OptimizeMetadataOnlyQuerySuite.scala
 ---
@@ -0,0 +1,122 @@
+/*
+ * 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
+
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.plans.logical.LocalRelation
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.test.SharedSQLContext
+
+class OptimizeMetadataOnlyQuerySuite extends QueryTest with 
SharedSQLContext {
+  import testImplicits._
+
+  override def beforeAll(): Unit = {
+super.beforeAll()
+val data = (1 to 10).map(i => (i, s"data-$i", i % 2, if ((i % 2) == 0) 
"even" else "odd"))
+  .toDF("col1", "col2", "partcol1", "partcol2")
+data.write.partitionBy("partcol1", 
"partcol2").mode("append").saveAsTable("srcpart")
+  }
+
+  override protected def afterAll(): Unit = {
+try {
+  sql("DROP TABLE IF EXISTS srcpart")
+} finally {
+  super.afterAll()
+}
+  }
+
+  private def assertMetadataOnlyQuery(df: DataFrame): Unit = {
+val localRelations = df.queryExecution.optimizedPlan.collect {
+  case l @ LocalRelation(_, _) => l
+}
+assert(localRelations.size == 1)
+  }
+
+  private def assertNotMetadataOnlyQuery(df: DataFrame): Unit = {
+val localRelations = df.queryExecution.optimizedPlan.collect {
+  case l @ LocalRelation(_, _) => l
+}
+assert(localRelations.size == 0)
+  }
+
+  private def testMetadataOnly(name: String, sqls: String*): Unit = {
+test(name) {
+  withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "true") {
+sqls.foreach { case q => assertMetadataOnlyQuery(sql(q)) }
+  }
+  withSQLConf(SQLConf.OPTIMIZER_METADATA_ONLY.key -> "false") {
+sqls.foreach { case q => assertNotMetadataOnlyQuery(sql(q)) }
+  }
+}
+  }
+
+  private def testNotMetadataOnly(name: String, sqls: String*): Unit = {
--- End diff --

When is this one used?


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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14012
  
**[Test build #62100 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62100/consoleFull)**
 for PR 14012 at commit 
[`9b2b5a8`](https://github.com/apache/spark/commit/9b2b5a86205e5ff020d264cb21ba48bbe0074790).
 * 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 #14112: [SPARK-16240][ML] Model loading backward compatibility f...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14112
  
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 #14112: [SPARK-16240][ML] Model loading backward compatibility f...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14112
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62108/
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 #14112: [SPARK-16240][ML] Model loading backward compatibility f...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14112
  
**[Test build #62108 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62108/consoleFull)**
 for PR 14112 at commit 
[`2b13262`](https://github.com/apache/spark/commit/2b1326258cf3daf436ec54103d34b6b9c0b39be5).
 * 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 #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70314737
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
--- End diff --

```scala
val table = formatTableName(name.table)
val tid = TableIdentifier(table)
name.database.isEmpty && isTemporaryTable(tid)
```
...is the same as...
```scala
isTemporaryTable(name)
```


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

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



[GitHub] spark issue #14086: [SPARK-16463][SQL] Support `truncate` option in Overwrit...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/14086
  
Thank you for guiding me. I hope this PR become more valuable in Spark, too.


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

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



[GitHub] spark issue #14101: [SPARK-16431] [ML] Add a unified method that accepts sin...

2016-07-11 Thread husseinhazimeh
Github user husseinhazimeh commented on the issue:

https://github.com/apache/spark/pull/14101
  
@rxin your feedback would be appreciated


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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/14086
  
That's a good point of view. Right. I agree with you, `DROP/CREATE isn't 
the best solution in all cases`. We can automatically check the schema 
**compatibility** in some way. 
Let me dig more on this issue. It seems not simple, but looks more valuable.


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

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14131
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62096/
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 #14131: [SPARK-16318][SQL] Implement all remaining xpath functio...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14131
  
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 #14131: [SPARK-16318][SQL] Implement all remaining xpath functio...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14131
  
**[Test build #62096 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62096/consoleFull)**
 for PR 14131 at commit 
[`ce53833`](https://github.com/apache/spark/commit/ce538330840cd97da169290759aa951f87e44ded).
 * 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 #14054: [SPARK-16226] [SQL] Weaken JDBC isolation level to avoid...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14054
  
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 #14054: [SPARK-16226] [SQL] Weaken JDBC isolation level to avoid...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14054
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62098/
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 #14054: [SPARK-16226] [SQL] Weaken JDBC isolation level to avoid...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14054
  
**[Test build #62098 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62098/consoleFull)**
 for PR 14054 at commit 
[`772dcc3`](https://github.com/apache/spark/commit/772dcc32d637f8790325b86161cc343d42497c76).
 * 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 #14114: [SPARK-16458][SQL] SessionCatalog should support `listCo...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/14114
  
@hvanhovell .
I rebased and updated one place for `isTemporaryTable`.
Thank you for in-depth review.


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

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



[GitHub] spark issue #13494: [SPARK-15752] [SQL] Optimize metadata only query that ha...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread lianhuiwang
Github user lianhuiwang commented on the issue:

https://github.com/apache/spark/pull/13494
  
@hvanhovell I have addressed some of your comments. Thanks. Could you look 
at again?


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

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



[GitHub] spark pull request #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70309422
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
--- End diff --

Hi, `isTemporaryTable` checks `tid.database.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 #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70309114
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
--- End diff --

Oops. @hvanhovell . I found the original reason.
The logic of `isTemporaryTable` only lookup `current database`. We should 
support the following. 
```
val m3 = intercept[AnalysisException] {
  catalog.getTableMetadata(TableIdentifier("view1", Some("default")))
}.getMessage
assert(m3.contains("Table or view 'view1' not found in database 
'default'"))
```


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

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



[GitHub] spark pull request #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70307717
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -425,10 +443,11 @@ class SessionCatalog(
   def tableExists(name: TableIdentifier): Boolean = synchronized {
 val db = formatDatabaseName(name.database.getOrElse(currentDb))
 val table = formatTableName(name.table)
-if (name.database.isDefined || !tempTables.contains(table)) {
-  externalCatalog.tableExists(db, table)
+if (name.database.isEmpty && tempTables.contains(table)) {
--- End diff --

Yep. What I mean I'll follow your advice. I misunderstood the function 
definition. :)


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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70307372
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -425,10 +443,11 @@ class SessionCatalog(
   def tableExists(name: TableIdentifier): Boolean = synchronized {
 val db = formatDatabaseName(name.database.getOrElse(currentDb))
 val table = formatTableName(name.table)
-if (name.database.isDefined || !tempTables.contains(table)) {
-  externalCatalog.tableExists(db, table)
+if (name.database.isEmpty && tempTables.contains(table)) {
--- End diff --

Huh? The definition is exactly the same:
```scala
def isTemporaryTable(name: TableIdentifier): Boolean = synchronized {
  name.database.isEmpty && tempTables.contains(formatTableName(name.table))
}
```
... and ...
```scala
val table = formatTableName(name.table)
if (name.database.isEmpty && tempTables.contains(table)) {
  ...
}
```



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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14138
  
**[Test build #62107 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62107/consoleFull)**
 for PR 14138 at commit 
[`48172d1`](https://github.com/apache/spark/commit/48172d1be903ab5760aec2e76c69dd17166c9660).


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

2016-07-11 Thread sethah
Github user sethah commented on the issue:

https://github.com/apache/spark/pull/12374
  
@MechCoder I addressed your comments. I updated the scala doc for 
`findSplitsForContinuousFeature` to reflect the return type. I think it's fine 
to simply fix the doc for now. Let me know if you see anything else.


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

2016-07-11 Thread GayathriMurali
Github user GayathriMurali commented on the issue:

https://github.com/apache/spark/pull/14112
  
@hhbyyh Thanks for helping out. Updated commit includes logic to include 
topicDistributionCol @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 pull request #12374: [SPARK-14610][ML] Remove superfluous split for co...

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70307082
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -137,14 +137,47 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
 {
   val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
 Map(), Set(),
-Array(3), Gini, QuantileStrategy.Sort,
+Array(2), Gini, QuantileStrategy.Sort,
 0, 0, 0.0, 0, 0
   )
   val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 
2).map(_.toDouble)
   val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
   assert(splits.length === 1)
   assert(splits(0) === 1.0)
 }
+
+// find splits for constant feature
+{
+  val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
+Map(), Set(),
+Array(3), Gini, QuantileStrategy.Sort,
+0, 0, 0.0, 0, 0
+  )
+  val featureSamples = Array(0, 0, 0).map(_.toDouble)
+  val featureSamplesEmpty = Array.empty[Double]
+  val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
+  assert(splits === Array[Double]())
+  val splitsEmpty =
--- End diff --

The iterable passed to `findSplitsForContinuousFeature` is the result of a 
`groupByKey`, so I don't see how this could ever happen. Still, if this method 
is used elsewhere ever in the future, it would fail with 
`java.lang.UnsupportedOperationException`. I'm open to changing this.


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

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



[GitHub] spark issue #13969: [SPARK-16284][SQL] Implement reflect SQL function

2016-07-11 Thread petermaxlee
Github user petermaxlee commented on the issue:

https://github.com/apache/spark/pull/13969
  
I submitted a new pull request https://github.com/apache/spark/pull/14138 
and in that version only static methods are supported.



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

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



[GitHub] spark pull request #14138: [SPARK-16284][SQL] Implement reflect SQL function

2016-07-11 Thread petermaxlee
GitHub user petermaxlee opened a pull request:

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

[SPARK-16284][SQL] Implement reflect SQL function

## What changes were proposed in this pull request?
This patch implements reflect SQL function, which can be used to invoke a 
Java method in SQL. Slightly different from Hive, this implementation requires 
the class name and the method name to be literals. This implementation also 
supports only a smaller number of data types, and requires the function to be 
static, as suggested by @rxin in #13969.

java_method is an alias for reflect, so this should also resolve 
SPARK-16277.

## How was this patch tested?
Added expression unit tests and an end-to-end test.


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

$ git pull https://github.com/petermaxlee/spark reflect-static

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

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


commit 4c9b975536877a4649706ef5e4e87a48a292668a
Author: petermaxlee 
Date:   2016-07-08T16:57:16Z

Rebase

commit 3fec1610c855b7595489ec3a635f0987dbb4701b
Author: petermaxlee 
Date:   2016-07-07T06:50:46Z

code review

commit 1c7bcbe4e3ab686bb581d5abe177ca1d1c66698d
Author: petermaxlee 
Date:   2016-07-08T18:19:58Z

code review

commit d24ca97a0ac85b54ea64b017343fe9eb51ecc02e
Author: petermaxlee 
Date:   2016-07-08T21:40:55Z

Merge remote-tracking branch 'apache/master' into reflect

commit 0ccade2485c7bb4bc767aab3f46e9f9e76eb2db8
Author: petermaxlee 
Date:   2016-07-11T17:50:12Z

Merge remote-tracking branch 'apache/master' into reflect-static

commit 48172d1be903ab5760aec2e76c69dd17166c9660
Author: petermaxlee 
Date:   2016-07-11T18:03:56Z

Handle only static functions.




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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70306857
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -425,10 +443,11 @@ class SessionCatalog(
   def tableExists(name: TableIdentifier): Boolean = synchronized {
 val db = formatDatabaseName(name.database.getOrElse(currentDb))
 val table = formatTableName(name.table)
-if (name.database.isDefined || !tempTables.contains(table)) {
-  externalCatalog.tableExists(db, table)
+if (name.database.isEmpty && tempTables.contains(table)) {
--- End diff --

I'll fix this, too.


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

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



[GitHub] spark issue #12374: [SPARK-14610][ML] Remove superfluous split for continuou...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/12374
  
**[Test build #62106 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62106/consoleFull)**
 for PR 12374 at commit 
[`3bb28fe`](https://github.com/apache/spark/commit/3bb28fe671b7f2aeac486ea8eb87809a997e6245).


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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70306434
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
--- End diff --

The only reason not to use that is just its `synchronized`.
But, I'll update according to your advice.


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

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



[GitHub] spark pull request #12374: [SPARK-14610][ML] Remove superfluous split for co...

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70306370
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -114,7 +114,7 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
   )
   val featureSamples = Array(1, 1, 2, 2, 2, 2, 2, 2, 2, 2, 
3).map(_.toDouble)
   val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
-  assert(splits.length === 3)
+  assert(splits.length === 2)
--- End diff --

Done.


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

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70306325
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/tree/impl/RandomForest.scala ---
@@ -712,17 +712,23 @@ private[spark] object RandomForest extends Logging {
 splitIndex += 1
   }
   // Find best split.
-  val (bestFeatureSplitIndex, bestFeatureGainStats) =
-Range(0, numSplits).map { case splitIdx =>
-  val leftChildStats = 
binAggregates.getImpurityCalculator(nodeFeatureOffset, splitIdx)
-  val rightChildStats =
-binAggregates.getImpurityCalculator(nodeFeatureOffset, 
numSplits)
-  rightChildStats.subtract(leftChildStats)
-  gainAndImpurityStats = 
calculateImpurityStats(gainAndImpurityStats,
-leftChildStats, rightChildStats, binAggregates.metadata)
-  (splitIdx, gainAndImpurityStats)
-}.maxBy(_._2.gain)
-  (splits(featureIndex)(bestFeatureSplitIndex), 
bestFeatureGainStats)
+  if (numSplits == 0) {
--- End diff --

I agree. I modified your suggestion to work with a view, so we don't 
allocate unnecessary memory.


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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70306040
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
--- End diff --

Oh. I see what you mean now.
Hmm.


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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70305902
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -425,10 +443,11 @@ class SessionCatalog(
   def tableExists(name: TableIdentifier): Boolean = synchronized {
 val db = formatDatabaseName(name.database.getOrElse(currentDb))
 val table = formatTableName(name.table)
-if (name.database.isDefined || !tempTables.contains(table)) {
-  externalCatalog.tableExists(db, table)
+if (name.database.isEmpty && tempTables.contains(table)) {
--- End diff --

The second one have `database` name.


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

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



[GitHub] spark pull request #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70305811
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -425,10 +443,11 @@ class SessionCatalog(
   def tableExists(name: TableIdentifier): Boolean = synchronized {
 val db = formatDatabaseName(name.database.getOrElse(currentDb))
 val table = formatTableName(name.table)
-if (name.database.isDefined || !tempTables.contains(table)) {
-  externalCatalog.tableExists(db, table)
+if (name.database.isEmpty && tempTables.contains(table)) {
--- End diff --

Yep. I thought like that at the first commit. But, there is a testcase 
violation. The following case should be allowed.
```
CREATE TEMPORARY VIEW t ...
CREATE VIEW t
```


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

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70305796
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
+  CatalogTable(
+identifier = tid,
+tableType = CatalogTableType.VIEW,
+storage = CatalogStorageFormat.empty,
+schema = tempTables(table).output.map { c =>
+  CatalogColumn(
+name = c.name,
+dataType = c.dataType.catalogString,
+nullable = c.nullable,
+comment = Option(c.name)
--- End diff --

The metadata can contain the comment, but it is a bit of a PITA to get out:
`if (c.metadata.contains("comment")) Some(c.metadata.getString("comment")) 
else None `

So I am fine with leaving this as it is...


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

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70305428
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -137,14 +137,47 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
 {
   val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
 Map(), Set(),
-Array(3), Gini, QuantileStrategy.Sort,
+Array(2), Gini, QuantileStrategy.Sort,
 0, 0, 0.0, 0, 0
   )
   val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 
2).map(_.toDouble)
   val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
   assert(splits.length === 1)
   assert(splits(0) === 1.0)
 }
+
+// find splits for constant feature
+{
+  val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
+Map(), Set(),
+Array(3), Gini, QuantileStrategy.Sort,
+0, 0, 0.0, 0, 0
+  )
+  val featureSamples = Array(0, 0, 0).map(_.toDouble)
+  val featureSamplesEmpty = Array.empty[Double]
+  val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
+  assert(splits === Array[Double]())
+  val splitsEmpty =
+RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, 
fakeMetadata, 0)
+  assert(splitsEmpty === Array[Double]())
+}
+  }
+
+  test("train with constant features") {
+val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0))
+val data = Array.fill(5)(lp)
+val rdd = sc.parallelize(data)
+val strategy = new OldStrategy(
+  OldAlgo.Classification,
+  Gini,
+  maxDepth = 2,
+  numClasses = 100,
+  maxBins = 100,
+  categoricalFeaturesInfo = Map(0 -> 2, 1 -> 5))
--- End diff --

I am using this to not only test continuous features, but categorical ones 
as well. I think we should leave it.


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

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



[GitHub] spark pull request #12374: [SPARK-14610][ML] Remove superfluous split for co...

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70305479
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -137,14 +137,47 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
 {
   val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
 Map(), Set(),
-Array(3), Gini, QuantileStrategy.Sort,
+Array(2), Gini, QuantileStrategy.Sort,
 0, 0, 0.0, 0, 0
   )
   val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 
2).map(_.toDouble)
   val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
   assert(splits.length === 1)
   assert(splits(0) === 1.0)
 }
+
+// find splits for constant feature
+{
+  val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
+Map(), Set(),
+Array(3), Gini, QuantileStrategy.Sort,
+0, 0, 0.0, 0, 0
+  )
+  val featureSamples = Array(0, 0, 0).map(_.toDouble)
+  val featureSamplesEmpty = Array.empty[Double]
+  val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
+  assert(splits === Array[Double]())
+  val splitsEmpty =
+RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, 
fakeMetadata, 0)
+  assert(splitsEmpty === Array[Double]())
+}
+  }
+
+  test("train with constant features") {
--- End diff --

This test is not specific to continuous 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 #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70305351
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala ---
@@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
*/
   @throws[AnalysisException]("table does not exist")
   override def listColumns(tableName: String): Dataset[Column] = {
--- End diff --

Thank you again, @hvanhovell 

1. Yep. That is the purpose of this PR, to make the contract consistent 
with other APIs.
2. The existence checking here is redundant because it call other 
`listColumns`. The callee will check that.


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

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



[GitHub] spark pull request #12374: [SPARK-14610][ML] Remove superfluous split for co...

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70305309
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -137,14 +137,47 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
 {
   val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
 Map(), Set(),
-Array(3), Gini, QuantileStrategy.Sort,
+Array(2), Gini, QuantileStrategy.Sort,
 0, 0, 0.0, 0, 0
   )
   val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 
2).map(_.toDouble)
   val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
   assert(splits.length === 1)
   assert(splits(0) === 1.0)
 }
+
+// find splits for constant feature
+{
+  val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
+Map(), Set(),
+Array(3), Gini, QuantileStrategy.Sort,
+0, 0, 0.0, 0, 0
+  )
+  val featureSamples = Array(0, 0, 0).map(_.toDouble)
+  val featureSamplesEmpty = Array.empty[Double]
+  val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
+  assert(splits === Array[Double]())
+  val splitsEmpty =
+RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, 
fakeMetadata, 0)
+  assert(splitsEmpty === Array[Double]())
+}
+  }
+
+  test("train with constant features") {
+val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0))
+val data = Array.fill(5)(lp)
+val rdd = sc.parallelize(data)
+val strategy = new OldStrategy(
+  OldAlgo.Classification,
+  Gini,
+  maxDepth = 2,
+  numClasses = 100,
--- End diff --

Not sure what this comment means, but if it's a reference to style, then 
the current syntax is correct. Unlike Python, named parameters are provided 
with spaces. See 
[here](http://docs.scala-lang.org/sips/completed/named-and-default-arguments.html)


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

2016-07-11 Thread sethah
Github user sethah commented on a diff in the pull request:

https://github.com/apache/spark/pull/12374#discussion_r70304950
  
--- Diff: 
mllib/src/test/scala/org/apache/spark/ml/tree/impl/RandomForestSuite.scala ---
@@ -137,14 +137,47 @@ class RandomForestSuite extends SparkFunSuite with 
MLlibTestSparkContext {
 {
   val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
 Map(), Set(),
-Array(3), Gini, QuantileStrategy.Sort,
+Array(2), Gini, QuantileStrategy.Sort,
 0, 0, 0.0, 0, 0
   )
   val featureSamples = Array(0, 1, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 2, 
2).map(_.toDouble)
   val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
   assert(splits.length === 1)
   assert(splits(0) === 1.0)
 }
+
+// find splits for constant feature
+{
+  val fakeMetadata = new DecisionTreeMetadata(1, 0, 0, 0,
+Map(), Set(),
+Array(3), Gini, QuantileStrategy.Sort,
+0, 0, 0.0, 0, 0
+  )
+  val featureSamples = Array(0, 0, 0).map(_.toDouble)
+  val featureSamplesEmpty = Array.empty[Double]
+  val splits = 
RandomForest.findSplitsForContinuousFeature(featureSamples, fakeMetadata, 0)
+  assert(splits === Array[Double]())
+  val splitsEmpty =
+RandomForest.findSplitsForContinuousFeature(featureSamplesEmpty, 
fakeMetadata, 0)
+  assert(splitsEmpty === Array[Double]())
+}
+  }
+
+  test("train with constant features") {
+val lp = LabeledPoint(1.0, Vectors.dense(0.0, 0.0, 0.0))
+val data = Array.fill(5)(lp)
+val rdd = sc.parallelize(data)
+val strategy = new OldStrategy(
+  OldAlgo.Classification,
+  Gini,
+  maxDepth = 2,
+  numClasses = 100,
+  maxBins = 100,
+  categoricalFeaturesInfo = Map(0 -> 2, 1 -> 5))
+val Array(tree) = RandomForest.run(rdd, strategy, 1, "all", 42L, instr 
= None)
+assert(tree.rootNode.impurity === -1.0)
--- End diff --

No, since the node found no valid split we flag the impurity as invalid. 
See 
[here](https://github.com/apache/spark/blob/master/mllib/src/main/scala/org/apache/spark/ml/tree/Node.scala#L287)


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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13758
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62103/
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 #13758: [SPARK-16043][SQL] Prepare GenericArrayData implementati...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13758
  
**[Test build #62103 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62103/consoleFull)**
 for PR 13758 at commit 
[`bd5b0d9`](https://github.com/apache/spark/commit/bd5b0d98ee05b663a1f43ecf88cfb3405d0c6a43).
 * 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 #13758: [SPARK-16043][SQL] Prepare GenericArrayData implementati...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13758
  
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 #13831: [SPARK-16119][sql] Support PURGE option to drop table / ...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70303041
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -246,9 +246,27 @@ class SessionCatalog(
   def getTableMetadata(name: TableIdentifier): CatalogTable = {
 val db = 
formatDatabaseName(name.database.getOrElse(getCurrentDatabase))
 val table = formatTableName(name.table)
-requireDbExists(db)
-requireTableExists(TableIdentifier(table, Some(db)))
-externalCatalog.getTable(db, table)
+val tid = TableIdentifier(table)
+if (name.database.isEmpty && isTemporaryTable(tid)) {
--- End diff --

`isTemporaryTable` also checks `name.database.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 #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70302921
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/SessionCatalog.scala
 ---
@@ -425,10 +443,11 @@ class SessionCatalog(
   def tableExists(name: TableIdentifier): Boolean = synchronized {
 val db = formatDatabaseName(name.database.getOrElse(currentDb))
 val table = formatTableName(name.table)
-if (name.database.isDefined || !tempTables.contains(table)) {
-  externalCatalog.tableExists(db, table)
+if (name.database.isEmpty && tempTables.contains(table)) {
--- End diff --

Shouldn't we use isTemporaryTable(...) here?


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

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



[GitHub] spark pull request #14132: [SPARK-16475][SQL][WIP] Broadcast Hint for SQL Qu...

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14132#discussion_r70302626
  
--- Diff: 
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
@@ -347,6 +347,14 @@ querySpecification
windows?)
 ;
 
+hint
+: '/*+' mapJoinHint '*/'
+;
+
+mapJoinHint
+: MAPJOIN '(' broadcastedTables+=tableIdentifier (',' 
broadcastedTables+=tableIdentifier)* ')'
--- End diff --

I see. Let me try 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 #13831: [SPARK-16119][sql] Support PURGE option to drop table / ...

2016-07-11 Thread vanzin
Github user vanzin commented on the issue:

https://github.com/apache/spark/pull/13831
  
I'd like to get this in soon, so pending tests, I'll leave this open until 
tomorrow and then push, unless I hear otherwise.


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

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70302158
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala ---
@@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
*/
   @throws[AnalysisException]("table does not exist")
   override def listColumns(tableName: String): Dataset[Column] = {
--- End diff --

Should't we check if the table exists? (like the other listColumns(...) 
function)


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

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



[GitHub] spark issue #13758: [SPARK-16043][SQL] Prepare GenericArrayData implementati...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14088
  
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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14088
  
Test PASSed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62101/
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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14088
  
**[Test build #62101 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62101/consoleFull)**
 for PR 14088 at commit 
[`bd7cf69`](https://github.com/apache/spark/commit/bd7cf696c11f4e74de9f96d104e4df6caa305357).
 * 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 #14114: [SPARK-16458][SQL] SessionCatalog should support ...

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on a diff in the pull request:

https://github.com/apache/spark/pull/14114#discussion_r70301759
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/internal/CatalogImpl.scala ---
@@ -138,7 +138,7 @@ class CatalogImpl(sparkSession: SparkSession) extends 
Catalog {
*/
   @throws[AnalysisException]("table does not exist")
   override def listColumns(tableName: String): Dataset[Column] = {
--- End diff --

This changes the contract of the `listColumns(...)` function. It now 
returns either a temporary view or a table in the current database. We have to 
document this! What happens when we have temporary table with the same name as 
a table in the current database?


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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on a diff in the pull request:

https://github.com/apache/spark/pull/14132#discussion_r70301194
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/parser/AstBuilder.scala
 ---
@@ -339,8 +339,24 @@ class AstBuilder extends SqlBaseBaseVisitor[AnyRef] 
with Logging {
   case SqlBaseParser.SELECT =>
 // Regular select
 
+// Broadcast hints
+var withBroadcastedTable = relation
+if (ctx.hint != null) {
+  val broadcastedTables =
+
hint.mapJoinHint.broadcastedTables.asScala.map(visitTableIdentifier)
+  for (table <- broadcastedTables) {
+var stop = false
+withBroadcastedTable = withBroadcastedTable.transformDown {
+  case r @ BroadcastHint(UnresolvedRelation(_, _)) => r
+  case r @ UnresolvedRelation(t, _) if !stop && t == table =>
--- End diff --

Thank you for review and confirm, @lianhuiwang !


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

2016-07-11 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/14132#discussion_r70300858
  
--- Diff: 
sql/catalyst/src/main/antlr4/org/apache/spark/sql/catalyst/parser/SqlBase.g4 ---
@@ -347,6 +347,14 @@ querySpecification
windows?)
 ;
 
+hint
+: '/*+' mapJoinHint '*/'
+;
+
+mapJoinHint
+: MAPJOIN '(' broadcastedTables+=tableIdentifier (',' 
broadcastedTables+=tableIdentifier)* ')'
--- End diff --

Can we move the logic somewhere else that's not in the parser rule? It'd be 
great if we don't need to modify the parser rule in order to support 
"broadcast" or "broadcast join". Even moving it into the parser itself (that is 
not parser rule could work).



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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14131
  
**[Test build #3179 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/3179/consoleFull)**
 for PR 14131 at commit 
[`ce53833`](https://github.com/apache/spark/commit/ce538330840cd97da169290759aa951f87e44ded).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14020: [SPARK-16349][sql] Fall back to isolated class loader wh...

2016-07-11 Thread vanzin
Github user vanzin commented on the issue:

https://github.com/apache/spark/pull/14020
  
I'll leave this open until EOD then push the change.


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

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



[GitHub] spark pull request #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get us...

2016-07-11 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/14088#discussion_r70299458
  
--- Diff: 
yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala ---
@@ -274,6 +288,37 @@ private object YarnClusterDriverWithFailure extends 
Logging with Matchers {
   }
 }
 
+private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging 
with Matchers {
+  def main(args: Array[String]): Unit = {
+if (args.length != 2) {
+  // scalastyle:off println
+  System.err.println(
+s"""
+|Invalid command line: ${args.mkString(" ")}
+|
+|Usage: YarnClusterDriverUseSparkHadoopUtilConf 
[propertyKey=value] [result file]
+""".stripMargin)
+  // scalastyle:on println
+  System.exit(1)
+}
+
+val sc = new SparkContext(new SparkConf()
+  .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+  .setAppName("yarn test using SparkHadoopUtil's conf"))
+
+val propertyKeyValue = args(0).split("=")
--- End diff --

Seems that you could pass `key` and `value` separately instead of having to 
parse the config; and that you don't need to pass `key` with the `spark.hadoop` 
prefix here either. You might even just hardcode the values instead...


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

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

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


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

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



[GitHub] spark pull request #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get us...

2016-07-11 Thread vanzin
Github user vanzin commented on a diff in the pull request:

https://github.com/apache/spark/pull/14088#discussion_r70299232
  
--- Diff: 
yarn/src/test/scala/org/apache/spark/deploy/yarn/YarnClusterSuite.scala ---
@@ -274,6 +288,37 @@ private object YarnClusterDriverWithFailure extends 
Logging with Matchers {
   }
 }
 
+private object YarnClusterDriverUseSparkHadoopUtilConf extends Logging 
with Matchers {
+  def main(args: Array[String]): Unit = {
+if (args.length != 2) {
+  // scalastyle:off println
+  System.err.println(
+s"""
+|Invalid command line: ${args.mkString(" ")}
+|
+|Usage: YarnClusterDriverUseSparkHadoopUtilConf 
[propertyKey=value] [result file]
+""".stripMargin)
+  // scalastyle:on println
+  System.exit(1)
+}
+
+val sc = new SparkContext(new SparkConf()
+  .set("spark.extraListeners", classOf[SaveExecutorInfo].getName)
+  .setAppName("yarn test using SparkHadoopUtil's conf"))
+
+val propertyKeyValue = args(0).split("=")
+val status = new File(args(1))
+var result = "failure"
+try {
+  SparkHadoopUtil.get.conf.get(propertyKeyValue(0).drop(13)) should be 
(propertyKeyValue(1))
--- End diff --

What is `drop(13)`? (hint: magic numbers are bad.)


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

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14045
  
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 #14045: [SPARK-16362][SQL][WIP] Support ArrayType and StructType...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14045
  
**[Test build #62094 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62094/consoleFull)**
 for PR 14045 at commit 
[`9a8b062`](https://github.com/apache/spark/commit/9a8b0629a60fecdd845e8632aad900865cc4229e).
 * 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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14088
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62099/
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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14088
  
**[Test build #62099 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62099/consoleFull)**
 for PR 14088 at commit 
[`352f852`](https://github.com/apache/spark/commit/352f8527e436a919e74b107de51c6dee24c63b73).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14088
  
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a program ...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread hvanhovell
Github user hvanhovell commented on the issue:

https://github.com/apache/spark/pull/14115
  
Cherry picked this into 2.0.


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

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



[GitHub] spark issue #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14012
  
**[Test build #62100 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62100/consoleFull)**
 for PR 14012 at commit 
[`9b2b5a8`](https://github.com/apache/spark/commit/9b2b5a86205e5ff020d264cb21ba48bbe0074790).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14088
  
**[Test build #62099 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62099/consoleFull)**
 for PR 14088 at commit 
[`352f852`](https://github.com/apache/spark/commit/352f8527e436a919e74b107de51c6dee24c63b73).


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

2016-07-11 Thread vectorijk
Github user vectorijk commented on a diff in the pull request:

https://github.com/apache/spark/pull/14136#discussion_r70294379
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/Percentile.scala
 ---
@@ -0,0 +1,148 @@
+/*
+ * 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.catalyst.expressions.aggregate
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.util.GenericArrayData
+import org.apache.spark.sql.types._
+import org.apache.spark.util.collection.OpenHashMap
+
+/**
+ * The Percentile aggregate function computes the exact percentile(s) of 
expr at pc with range in
+ * [0, 1].
+ * The parameter pc can be a DoubleType or DoubleType array.
+ */
+@ExpressionDescription(
+  usage = """_FUNC_(epxr, pc) - Returns the percentile(s) of expr at pc 
(range: [0,1]). pc can be
+  a double or double array.""")
+case class Percentile(
+ child: Expression,
+ pc: Seq[Double],
+ mutableAggBufferOffset: Int = 0,
+ inputAggBufferOffset: Int = 0)
+  extends ImperativeAggregate {
+
+  def this(child: Expression, pc: Double) = {
+this(child = child, pc = Seq(pc), mutableAggBufferOffset = 0, 
inputAggBufferOffset = 0)
+  }
+
+  override def prettyName: String = "percentile"
+
+  override def withNewMutableAggBufferOffset(newMutableAggBufferOffset: 
Int): ImperativeAggregate =
+copy(mutableAggBufferOffset = newMutableAggBufferOffset)
+
+  override def withNewInputAggBufferOffset(newInputAggBufferOffset: Int): 
ImperativeAggregate =
+copy(inputAggBufferOffset = newInputAggBufferOffset)
+
+  var counts = new OpenHashMap[Long, Long]()
--- End diff --

@hvanhovell Thanks!


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

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



[GitHub] spark issue #14130: [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT

2016-07-11 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/14130
  
OK, later we'd have to make the base version 2.0.0 right? that's fine 
though either way we have to bump something later, so was going to wait, but, 
no big deal either way, especially if it reduces confusion about the snapshot 
number


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

2016-07-11 Thread asfgit
Github user asfgit closed the pull request at:

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


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

2016-07-11 Thread jiangxb1987
Github user jiangxb1987 commented on the issue:

https://github.com/apache/spark/pull/14136
  
@hvanhovell Thank you for your kindly review, the suggestions are quite 
useful for me. I'll try to get some time later today to update some fixes. 
Thanks!


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

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



[GitHub] spark issue #14054: [SPARK-16226] [SQL] Weaken JDBC isolation level to avoid...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14054
  
**[Test build #62098 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62098/consoleFull)**
 for PR 14054 at commit 
[`772dcc3`](https://github.com/apache/spark/commit/772dcc32d637f8790325b86161cc343d42497c76).


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

2016-07-11 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/14130
  
I'm going to merge this in master.



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

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



[GitHub] spark issue #14130: [SPARK-16477] Bump master version to 2.1.0-SNAPSHOT

2016-07-11 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/14130
  
Yup once we actually release we can bump the base version. The good thing 
with the new approach is that we can now define mima excludes based on 2.0 or 
2.1 versions already.



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

2016-07-11 Thread dongjoon-hyun
Github user dongjoon-hyun commented on the issue:

https://github.com/apache/spark/pull/14115
  
If pissible, I hope. :)


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

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13704
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62097/
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a program ...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/13704
  
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 #13704: [SPARK-15985][SQL] Reduce runtime overhead of a program ...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/13704
  
**[Test build #62097 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62097/consoleFull)**
 for PR 13704 at commit 
[`b7477de`](https://github.com/apache/spark/commit/b7477de4c79dac42b42d745e419654dcf831bdba).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class SimplifyCastsSuite extends PlanTest `


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

2016-07-11 Thread srowen
Github user srowen commented on the issue:

https://github.com/apache/spark/pull/14054
  
@rxin I added configurability; does that look reasonable? there are several 
ways to play this one.


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

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



[GitHub] spark issue #13704: [SPARK-15985][SQL] Reduce runtime overhead of a program ...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14088
  
Test FAILed.
Refer to this link for build results (access rights to CI server needed): 
https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/62095/
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 #14131: [SPARK-16318][SQL] Implement all remaining xpath functio...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

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


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

2016-07-11 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/14130
  
That's why I did https://github.com/apache/spark/pull/14128

Base mima version is still 1.6, and 2.0 and 2.1 currently has the same set 
of excludes.



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA 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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread AmplabJenkins
Github user AmplabJenkins commented on the issue:

https://github.com/apache/spark/pull/14088
  
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 #14088: [SPARK-16414] [YARN] Fix bugs for "Can not get user conf...

2016-07-11 Thread SparkQA
Github user SparkQA commented on the issue:

https://github.com/apache/spark/pull/14088
  
**[Test build #62095 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/62095/consoleFull)**
 for PR 14088 at commit 
[`a90fe48`](https://github.com/apache/spark/commit/a90fe481fe50a4d4e67320fad6628395493d4a57).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark issue #14115: [SPARK-16459][SQL] Prevent dropping current database

2016-07-11 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/14115
  
Should this be in 2.0?



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

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



[GitHub] spark issue #14131: [SPARK-16318][SQL] Implement all remaining xpath functio...

2016-07-11 Thread petermaxlee
Github user petermaxlee commented on the issue:

https://github.com/apache/spark/pull/14131
  
I've added the checkEvaluation fix here too.



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

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



[GitHub] spark issue #14115: [SPARK-16459][SQL] Prevent dropping current database

2016-07-11 Thread rxin
Github user rxin commented on the issue:

https://github.com/apache/spark/pull/14115
  
@hvanhovell please leave a message on the pr next time when you merge one 
-- and also indicate the branch this was merged in (master in this case).



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

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



[GitHub] spark issue #14135: [Spark-16479] Add Example for asynchronous action

2016-07-11 Thread phalodi
Github user phalodi commented on the issue:

https://github.com/apache/spark/pull/14135
  
@srowen yeah you are right i will close this request can you please look 
into documentation pull request https://github.com/apache/spark/pull/14104


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

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



<    1   2   3   4   5   6   7   >