[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7954#discussion_r36385979
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/aggregate/TungstenAggregate.scala
 ---
@@ -0,0 +1,102 @@
+/*
+ * 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.aggregate
+
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.errors._
+import 
org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression2
+import org.apache.spark.sql.catalyst.expressions._
+import 
org.apache.spark.sql.catalyst.plans.physical.{UnspecifiedDistribution, 
ClusteredDistribution, AllTuples, Distribution}
+import org.apache.spark.sql.execution.{UnaryNode, SparkPlan}
+
+case class TungstenAggregate(
+requiredChildDistributionExpressions: Option[Seq[Expression]],
+groupingExpressions: Seq[NamedExpression],
+nonCompleteAggregateExpressions: Seq[AggregateExpression2],
+completeAggregateExpressions: Seq[AggregateExpression2],
+initialInputBufferOffset: Int,
+resultExpressions: Seq[NamedExpression],
+child: SparkPlan)
+  extends UnaryNode {
+
+  override def outputsUnsafeRows: Boolean = true
+
+  override def canProcessUnsafeRows: Boolean = true
+
+  override def canProcessSafeRows: Boolean = false
+
+  override def output: Seq[Attribute] = 
resultExpressions.map(_.toAttribute)
+
+  override def requiredChildDistribution: List[Distribution] = {
+requiredChildDistributionExpressions match {
+  case Some(exprs) if exprs.length == 0 => AllTuples :: Nil
+  case Some(exprs) if exprs.length > 0 => ClusteredDistribution(exprs) 
:: Nil
+  case None => UnspecifiedDistribution :: Nil
+}
+  }
+
+  // This is for testing. We force TungstenAggregationIterator to fall 
back to sort-based
+  // aggregation once it has processed a given number of input rows.
+  private val testFallbackStartsAt: Option[Int] = {
+sqlContext.getConf("spark.sql.TungstenAggregate.testFallbackStartsAt", 
null) match {
+  case null | "" => None
+  case fallbackStartsAt => Some(fallbackStartsAt.toInt)
+}
+  }
+
+  protected override def doExecute(): RDD[InternalRow] = attachTree(this, 
"execute") {
+child.execute().mapPartitions { iter =>
+  val hasInput = iter.hasNext
+  if (!hasInput && groupingExpressions.nonEmpty) {
+// This is a grouped aggregate and the input iterator is empty,
+// so return an empty iterator.
+Iterator.empty.asInstanceOf[Iterator[UnsafeRow]]
+  } else {
+val aggregationIterator =
+  new TungstenAggregationIterator(
+groupingExpressions,
+nonCompleteAggregateExpressions,
+completeAggregateExpressions,
+initialInputBufferOffset,
+resultExpressions,
+newMutableProjection,
+child.output,
+iter.asInstanceOf[Iterator[UnsafeRow]],
+testFallbackStartsAt)
+
+if (!hasInput && groupingExpressions.isEmpty) {
+  
Iterator.single[UnsafeRow](aggregationIterator.outputForEmptyGroupingKeyWithoutInput())
--- End diff --

you need to free the map, don't you?


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

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



[GitHub] spark pull request: [SPARK-8505][SparkR] Add settings to kick `lin...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7883#issuecomment-128271200
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-8625] [Core] Propagate user exceptions ...

2015-08-05 Thread tomwhite
Github user tomwhite commented on the pull request:

https://github.com/apache/spark/pull/7014#issuecomment-128271252
  
I rebased this on master. Is there a way to get this to be retested?


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

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



[GitHub] spark pull request: [SPARK-8505][SparkR] Add settings to kick `lin...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7883#issuecomment-128271162
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128271153
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9563][SQL] Remove repartition operators...

2015-08-05 Thread viirya
Github user viirya commented on the pull request:

https://github.com/apache/spark/pull/7959#issuecomment-128271116
  
To optimise `repartition(10).repartition(100)` to `repartition(100)` seems 
reasonable. Is it also doable for `repartition(100).repartition(10)` to 
`repartition(10)`?


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

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



[GitHub] spark pull request: [SPARK-9563][SQL] Remove repartition operators...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7959#issuecomment-128271181
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128271180
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9563][SQL] Remove repartition operators...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7959#issuecomment-128271157
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9556] [SPARK-9619] [SPARK-9624] [STREAM...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7913#issuecomment-128270370
  
  [Test build #39986 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39986/console)
 for   PR 7913 at commit 
[`41d4461`](https://github.com/apache/spark/commit/41d4461ec2d7bf5de6b35ba5001bce7bdcad95a8).
 * This patch **fails PySpark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request: [SPARK-9556] [SPARK-9619] [SPARK-9624] [STREAM...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7913#issuecomment-128270437
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128270068
  
  [Test build #39998 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39998/consoleFull)
 for   PR 7709 at commit 
[`9418088`](https://github.com/apache/spark/commit/9418088447d899e7b5ea00c167ddd85a4eb16b15).


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

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



[GitHub] spark pull request: [SPARK-9533] [PySpark] [ML] Add missing method...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7930#issuecomment-128270030
  
  [Test build #3 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/3/consoleFull)
 for   PR 7930 at commit 
[`1bea394`](https://github.com/apache/spark/commit/1bea394efe953098b832e083d87f1765b40d25f9).


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

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



[GitHub] spark pull request: [SPARK-9459] [SQL] use generated FromUnsafePro...

2015-08-05 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7840#discussion_r36385432
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -127,6 +127,8 @@ class CodeGenContext {
 dataType match {
   case _ if isPrimitiveType(jt) => 
s"$row.set${primitiveTypeName(jt)}($ordinal, $value)"
   case t: DecimalType => s"$row.setDecimal($ordinal, $value, 
${t.precision})"
+  // The UTF8String may came from UnsafeRow, otherwise clone is cheap 
(re-use the bytes)
+  case StringType => s"$row.update($ordinal, $value.clone())"
--- End diff --

Yes


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128269789
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128269778
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9533] [PySpark] [ML] Add missing method...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7930#issuecomment-128269791
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9533] [PySpark] [ML] Add missing method...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7930#issuecomment-128269776
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9533] [PySpark] [ML] Add missing method...

2015-08-05 Thread MechCoder
Github user MechCoder commented on the pull request:

https://github.com/apache/spark/pull/7930#issuecomment-128269563
  
@jkbradley Actally `findSynonyms` should be a method having parameters. Let 
me know if there is 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 pull request: [SPARK-9459] [SQL] use generated FromUnsafePro...

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

https://github.com/apache/spark/pull/7840#discussion_r36385277
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/CodeGenerator.scala
 ---
@@ -127,6 +127,8 @@ class CodeGenContext {
 dataType match {
   case _ if isPrimitiveType(jt) => 
s"$row.set${primitiveTypeName(jt)}($ordinal, $value)"
   case t: DecimalType => s"$row.setDecimal($ordinal, $value, 
${t.precision})"
+  // The UTF8String may came from UnsafeRow, otherwise clone is cheap 
(re-use the bytes)
+  case StringType => s"$row.update($ordinal, $value.clone())"
--- End diff --

But if the user-side don't cache anything, just go through the unsafe row 
and do come computation, we don't need to deep copy UTF8String inside unsafe 
row then?


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7977#discussion_r36385264
  
--- Diff: python/pyspark/sql/column.py ---
@@ -254,12 +255,29 @@ def inSet(self, *cols):
 [Row(age=5, name=u'Bob')]
 >>> df[df.age.inSet([1, 2, 3])].collect()
 [Row(age=2, name=u'Alice')]
+
+.. note:: Deprecated in 1.5, use :func:`Column.isin` instead.
+"""
+warnings.warn("inSet is deprecated. Use isin() instead.")
+self.isin(*cols)
--- End diff --

return


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

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



[GitHub] spark pull request: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128266964
  
  [Test build #39997 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39997/consoleFull)
 for   PR 7536 at commit 
[`5623370`](https://github.com/apache/spark/commit/5623370dbf805512db8ea61b719da87e62457ff3).


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

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



[GitHub] spark pull request: [SPARK-9637][Core] Add interface for implement...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7958#issuecomment-128266496
  
  [Test build #39996 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39996/consoleFull)
 for   PR 7958 at commit 
[`8574439`](https://github.com/apache/spark/commit/857443947f373e3335ea8ef64f50c3a73c762c0b).


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

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



[GitHub] spark pull request: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128266447
  
Merged build started.


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

2015-08-05 Thread srowen
Github user srowen commented on the pull request:

https://github.com/apache/spark/pull/7984#issuecomment-128266416
  
This is too trivial to open a PR for; it's not worth your / Jenkins' / 
committer time to bother with 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: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128266428
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128266349
  
  [Test build #244 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SlowSparkPullRequestBuilder/244/consoleFull)
 for   PR 7536 at commit 
[`5623370`](https://github.com/apache/spark/commit/5623370dbf805512db8ea61b719da87e62457ff3).


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

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



[GitHub] spark pull request: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128266282
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128266269
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9637][Core] Add interface for implement...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7958#issuecomment-128265854
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9189][CORE] Takes locality and the sum ...

2015-08-05 Thread watermen
Github user watermen commented on the pull request:

https://github.com/apache/spark/pull/7536#issuecomment-128265855
  
retest this please


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128265648
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-9637][Core] Add interface for implement...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7958#issuecomment-128265835
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128265584
  
  [Test build #39989 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39989/console)
 for   PR 7977 at commit 
[`2197d37`](https://github.com/apache/spark/commit/2197d379d350e256a87f9647b778dcd9509e1f75).
 * This patch **fails PySpark unit tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request: [SPARK-9556] [SPARK-9619] [SPARK-9624] [STREAM...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7913#issuecomment-128263375
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128263540
  
  [Test build #39995 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39995/consoleFull)
 for   PR 7952 at commit 
[`05e2216`](https://github.com/apache/spark/commit/05e22165d109b71d5b10374b6f39a744fa70050c).


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

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



[GitHub] spark pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128263261
  
  [Test build #243 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SlowSparkPullRequestBuilder/243/consoleFull)
 for   PR 7952 at commit 
[`05e2216`](https://github.com/apache/spark/commit/05e22165d109b71d5b10374b6f39a744fa70050c).


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

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



[GitHub] spark pull request: [SPARK-9556] [SPARK-9619] [SPARK-9624] [STREAM...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7913#issuecomment-128263311
  
  [Test build #242 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SlowSparkPullRequestBuilder/242/console)
 for   PR 7913 at commit 
[`41d4461`](https://github.com/apache/spark/commit/41d4461ec2d7bf5de6b35ba5001bce7bdcad95a8).
 * 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 pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128263186
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128263205
  
@zhichao-li Thanks for working on this, it's very closing, will merge it 
once you address the last comment.


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

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



[GitHub] spark pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128263194
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7709#discussion_r36384134
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -349,6 +351,78 @@ case class EndsWith(left: Expression, right: 
Expression)
   }
 }
 
+object StringTranslate {
+
+  def buildDict(matchingString: UTF8String, replaceString: UTF8String)
+: JMap[Character, Character] = {
+val matching = matchingString.toString()
+val replace = replaceString.toString()
+val dict = new HashMap[Character, Character]()
+var i = 0
+while (i < matching.length()) {
+  val rep = if (i < replace.length()) replace.charAt(i) else '\0'
+  if (null == dict.get(matching.charAt(i))) {
+dict.put(matching.charAt(i), rep)
+  }
+  i += 1
+}
+dict
+  }
+}
+
+/**
+ * A function translate any character in the `srcExpr` by a character in 
`replaceExpr`.
+ * The characters in `replaceExpr` is corresponding to the characters in 
`matchingExpr`.
+ * The translate will happen when any character in the string matching 
with the character
+ * in the `matchingExpr`.
+ */
+case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, 
replaceExpr: Expression)
+  extends TernaryExpression with ImplicitCastInputTypes {
+
+  @transient private var lastMatching: UTF8String = _
+  @transient private var lastReplace: UTF8String = _
+  @transient private var dict: JMap[Character, Character] = _
+
+  override def nullSafeEval(srcEval: Any, matchingEval: Any, replaceEval: 
Any): Any = {
+if (matchingEval != lastMatching || replaceEval != lastReplace) {
+  lastMatching = matchingEval.asInstanceOf[UTF8String].clone()
+  lastReplace = replaceEval.asInstanceOf[UTF8String].clone()
+  dict = StringTranslate.buildDict(lastMatching, lastReplace)
+}
+srcEval.asInstanceOf[UTF8String].translate(dict)
+  }
+
+  override def genCode(ctx: CodeGenContext, ev: GeneratedExpressionCode): 
String = {
+val termLastMatching = ctx.freshName("lastMatching")
+val termLastReplace = ctx.freshName("lastReplace")
+val termDict = ctx.freshName("dict")
+val classNameDict = classOf[JMap[Character, 
Character]].getCanonicalName
+
+ctx.addMutableState("UTF8String", termLastMatching, 
s"${termLastMatching} = null;")
+ctx.addMutableState("UTF8String", termLastReplace, 
s"${termLastReplace} = null;")
+ctx.addMutableState(classNameDict, termDict, s"${termDict} = null;")
+
+nullSafeCodeGen(ctx, ev, (src, matching, replace) => {
+  s"""if (!(${matchingExpr.foldable} && ${replaceExpr.foldable} && 
${termDict} != null) ||
--- End diff --

This is confusing, when matchingExpr.foldable is true and is true, it will 
still check !${matching}.equals(${termLastMatching})

```
val check = if (matchingExpr.foldable && replaceExpr.foldable) {
 s"${termDict} == null"
} else {
  s"!${matching}.equals(${termLastMatching}) || 
!${replace}.equals(${termLastReplace})"
}

s"""
if ($check) {
 ...
}
"""
```


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

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



[GitHub] spark pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128262781
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128262793
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9493] [ML] add featureIndex to handle v...

2015-08-05 Thread jkbradley
Github user jkbradley commented on the pull request:

https://github.com/apache/spark/pull/7952#issuecomment-128262626
  
Jenkins test this please


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

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



[GitHub] spark pull request: [SPARK-6212][SQL]The EXPLAIN output of CTAS on...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7986#issuecomment-128262060
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7978#issuecomment-128262031
  
  [Test build #1376 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1376/consoleFull)
 for   PR 7978 at commit 
[`923c9eb`](https://github.com/apache/spark/commit/923c9ebe2c8da057eecb2f18f307013676e4edb6).


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

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



[GitHub] spark pull request: [SPARK-6212][SQL]The EXPLAIN output of CTAS on...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7986#issuecomment-128262007
  
  [Test build #39987 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39987/console)
 for   PR 7986 at commit 
[`bd7eb20`](https://github.com/apache/spark/commit/bd7eb206ef43d3b2b29ab34fd37a5f869aba094f).
 * 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 pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7978#issuecomment-128261384
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7978#issuecomment-128261695
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7978#issuecomment-128261501
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9482] [SQL] Fix thread-safey issue of u...

2015-08-05 Thread davies
Github user davies commented on the pull request:

https://github.com/apache/spark/pull/7940#issuecomment-128261671
  
The semijoin test had ran 100 times without failure, so we could think that 
is fixed. I'm going to remove the debug code, the fix others.


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

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



[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7954#discussion_r36383339
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
 ---
@@ -58,11 +58,26 @@ private class UnsafeRowSerializerInstance(numFields: 
Int) extends SerializerInst
*/
   override def serializeStream(out: OutputStream): SerializationStream = 
new SerializationStream {
 private[this] var writeBuffer: Array[Byte] = new Array[Byte](4096)
+// When `out` is backed by ChainedBufferOutputStream, we will get an
+// UnsupportedOperationException when we call dOut.writeInt because it 
internally calls
+// ChainedBufferOutputStream's write(b: Int), which is not supported.
+// To workaround this issue, we create an array for sorting the int 
value.
+// To reproduce the problem, use dOut.writeInt(row.getSizeInBytes) and
+// run SparkSqlSerializer2SortMergeShuffleSuite.
+private[this] var intBuffer: Array[Byte] = new Array[Byte](4)
 private[this] val dOut: DataOutputStream = new DataOutputStream(out)
 
 override def writeValue[T: ClassTag](value: T): SerializationStream = {
   val row = value.asInstanceOf[UnsafeRow]
-  dOut.writeInt(row.getSizeInBytes)
+  val size = row.getSizeInBytes
+  // This part is based on DataOutputStream's writeInt.
+  // It is for dOut.writeInt(row.getSizeInBytes).
+  intBuffer(0) = ((size >>> 24) & 0xFF).toByte
+  intBuffer(1) = ((size >>> 16) & 0xFF).toByte
+  intBuffer(2) = ((size >>> 8) & 0xFF).toByte
+  intBuffer(3) = ((size >>> 0) & 0xFF).toByte
+  dOut.write(intBuffer, 0, 4)
--- End diff --

cc @JoshRosen do you think this is fine? seems inefficient to me but maybe 
there is no better way


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

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



[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7954#discussion_r36383162
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/aggregate/functions.scala
 ---
@@ -256,12 +256,16 @@ case class Sum(child: Expression) extends 
AlgebraicAggregate {
   override def dataType: DataType = resultType
 
   // Expected input data type.
+  // TODO: Once we remove the old code path, we can use our analyzer to 
cast NullType
--- End diff --

which old path are we talking about? the "old" aggregate code path is not 
using sum here, is 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: [SPARK-9632][SQL] update InternalRow.toSeq to ...

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

https://github.com/apache/spark/pull/7955#discussion_r36383134
  
--- Diff: 
sql/hive/src/main/scala/org/apache/spark/sql/hive/hiveWriterContainers.scala ---
@@ -211,18 +211,18 @@ private[spark] class 
SparkHiveDynamicPartitionWriterContainer(
   }
 }
 
-val dynamicPartPath = dynamicPartColNames
-  .zip(row.toSeq.takeRight(dynamicPartColNames.length))
-  .map { case (col, rawVal) =>
-val string = if (rawVal == null) null else 
convertToHiveRawString(col, rawVal)
-val colString =
-  if (string == null || string.isEmpty) {
-defaultPartName
-  } else {
-FileUtils.escapePathName(string, defaultPartName)
-  }
-s"/$col=$colString"
-  }.mkString
+val nonDynamicPartLen = schema.length - dynamicPartColNames.length
+val dynamicPartPath = dynamicPartColNames.zipWithIndex.map { case 
(colName, i) =>
+  val rawVal = row.get(nonDynamicPartLen + i, schema(colName).dataType)
--- End diff --

Still got `ArrayIndexOutOfBoundsException` here, what's the relationship 
between `dynamicPartColNames`, `row` and `schema`?


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

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



[GitHub] spark pull request: [SPARK-8505][SparkR] Add settings to kick `lin...

2015-08-05 Thread yu-iskw
Github user yu-iskw commented on the pull request:

https://github.com/apache/spark/pull/7883#issuecomment-128252880
  
@shaneknapp thanks!


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

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



[GitHub] spark pull request: [SPARK-9632][SQL] update InternalRow.toSeq to ...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7955#issuecomment-128251984
  
  [Test build #39985 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39985/console)
 for   PR 7955 at commit 
[`4addf29`](https://github.com/apache/spark/commit/4addf29152bad5f257bf833dc6e96f5886704c69).
 * This patch **fails Spark unit tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `final class SpecificMutableRow(val values: Array[MutableValue])`
  * `  final class SpecificRow extends $`
  * `trait BaseGenericInternalRow extends InternalRow `
  * `class GenericMutableRow(values: Array[Any]) extends MutableRow with 
BaseGenericInternalRow `



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

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



[GitHub] spark pull request: [SPARK-9632][SQL] update InternalRow.toSeq to ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7955#issuecomment-128252020
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SQL] [WIP] Join test coverage improvements

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7985#issuecomment-128250757
  
  [Test build #39993 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39993/consoleFull)
 for   PR 7985 at commit 
[`2048cad`](https://github.com/apache/spark/commit/2048cad3e30a1cbb5317bbc6c6a9c2891de3abb7).


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

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



[GitHub] spark pull request: [SPARK-8978][Streaming] Implements the DirectK...

2015-08-05 Thread nraychaudhuri
Github user nraychaudhuri commented on the pull request:

https://github.com/apache/spark/pull/7796#issuecomment-128250793
  
Not sure why pySpark tests passed and failed for the same commit. Trying to 
build again


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

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



[GitHub] spark pull request: [SPARK-8978][Streaming] Implements the DirectK...

2015-08-05 Thread nraychaudhuri
Github user nraychaudhuri commented on the pull request:

https://github.com/apache/spark/pull/7796#issuecomment-128250803
  
Jenkins, test this please.


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on the pull request:

https://github.com/apache/spark/pull/7978#issuecomment-128250117
  
LGTM other than that.



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

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



[GitHub] spark pull request: [SPARK-7165] [WIP] [SQL] Use sort merge join f...

2015-08-05 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/7904#issuecomment-128250099
  
This is temporarily blocked on a smaller patch to greatly improve the unit 
test coverage of our join operators (see #7985).


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

2015-08-05 Thread JoshRosen
Github user JoshRosen commented on the pull request:

https://github.com/apache/spark/pull/7985#issuecomment-128250024
  
My goal here is to keep adding tests until I hit 100% line coverage (and 
ideally 100% branch coverage) on SortMergeJoin. 


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

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7985#issuecomment-128250070
  
 Merged build triggered.


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

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7985#issuecomment-128250093
  
Merged build started.


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

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7985#issuecomment-128249338
  
Merged build finished. Test PASSed.


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

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



[GitHub] spark pull request: [SQL] [WIP] Join test coverage improvements

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7985#issuecomment-128249213
  
  [Test build #39984 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39984/console)
 for   PR 7985 at commit 
[`be61fbe`](https://github.com/apache/spark/commit/be61fbe370aa02fa471ec918cb8b5f5987c033d6).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request: [SPARK-9674][SPARK-9667] Remove SparkSqlSerial...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128249141
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-9674][SPARK-9667] Remove SparkSqlSerial...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128249071
  
  [Test build #39990 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39990/console)
 for   PR 7981 at commit 
[`82d1dc7`](https://github.com/apache/spark/commit/82d1dc73a05d4b5605acac02f9f0ebce1ecdf6f0).
 * 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 pull request: [SPARK-9632][SQL] update InternalRow.toSeq to ...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7955#issuecomment-128248723
  
Merged build finished. Test FAILed.


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

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



[GitHub] spark pull request: [SPARK-9632][SQL] update InternalRow.toSeq to ...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7955#issuecomment-128248588
  
  [Test build #241 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SlowSparkPullRequestBuilder/241/console)
 for   PR 7955 at commit 
[`4addf29`](https://github.com/apache/spark/commit/4addf29152bad5f257bf833dc6e96f5886704c69).
 * 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 pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7709#discussion_r36382246
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -349,6 +351,78 @@ case class EndsWith(left: Expression, right: 
Expression)
   }
 }
 
+object StringTranslate {
+
+  def buildDict(matchingString: UTF8String, replaceString: UTF8String)
+: JMap[Character, Character] = {
+val matching = matchingString.toString()
+val replace = replaceString.toString()
+val dict = new HashMap[Character, Character]()
+var i = 0
+while (i < matching.length()) {
+  val rep = if (i < replace.length()) replace.charAt(i) else '\0'
+  if (null == dict.get(matching.charAt(i))) {
+dict.put(matching.charAt(i), rep)
+  }
+  i += 1
+}
+dict
+  }
+}
+
+/**
+ * A function translate any character in the `srcExpr` by a character in 
`replaceExpr`.
+ * The characters in `replaceExpr` is corresponding to the characters in 
`matchingExpr`.
+ * The translate will happen when any character in the string matching 
with the character
+ * in the `matchingExpr`.
+ */
+case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, 
replaceExpr: Expression)
+  extends TernaryExpression with ImplicitCastInputTypes {
+
+  @transient private var lastMatching: UTF8String = _
+  @transient private var lastReplace: UTF8String = _
+  @transient private var dict: JMap[Character, Character] = _
+
+  override def nullSafeEval(srcEval: Any, matchingEval: Any, replaceEval: 
Any): Any = {
+if (matchingEval != lastMatching || replaceEval != lastReplace) {
+  lastMatching = matchingEval.asInstanceOf[UTF8String]
--- End diff --

We recently changed to do not copy the bytes when call 
UnsafeRow.getUTF8String(), so whenever we want to hold a UTF8String, we need to 
do a copy.


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

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



[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread yhuai
Github user yhuai commented on a diff in the pull request:

https://github.com/apache/spark/pull/7954#discussion_r36382000
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/UnsafeRowSerializer.scala
 ---
@@ -58,11 +58,26 @@ private class UnsafeRowSerializerInstance(numFields: 
Int) extends SerializerInst
*/
   override def serializeStream(out: OutputStream): SerializationStream = 
new SerializationStream {
 private[this] var writeBuffer: Array[Byte] = new Array[Byte](4096)
+// When `out` is backed by ChainedBufferOutputStream, we will get an
+// UnsupportedOperationException when we call dOut.writeInt because it 
internally calls
+// ChainedBufferOutputStream's write(b: Int), which is not supported.
+// To workaround this issue, we create an array for sorting the int 
value.
+// To reproduce the problem, use dOut.writeInt(row.getSizeInBytes) and
+// run SparkSqlSerializer2SortMergeShuffleSuite.
+private[this] var intBuffer: Array[Byte] = new Array[Byte](4)
 private[this] val dOut: DataOutputStream = new DataOutputStream(out)
 
 override def writeValue[T: ClassTag](value: T): SerializationStream = {
   val row = value.asInstanceOf[UnsafeRow]
-  dOut.writeInt(row.getSizeInBytes)
+  val size = row.getSizeInBytes
+  // This part is based on DataOutputStream's writeInt.
+  // It is for dOut.writeInt(row.getSizeInBytes).
+  intBuffer(0) = ((size >>> 24) & 0xFF).toByte
+  intBuffer(1) = ((size >>> 16) & 0xFF).toByte
+  intBuffer(2) = ((size >>> 8) & 0xFF).toByte
+  intBuffer(3) = ((size >>> 0) & 0xFF).toByte
+  dOut.write(intBuffer, 0, 4)
--- End diff --

Also, we need to double check if we need to wrap input stream with a 
buffered input stream when we read data back.


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread zhichao-li
Github user zhichao-li commented on a diff in the pull request:

https://github.com/apache/spark/pull/7709#discussion_r36381923
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -349,6 +351,78 @@ case class EndsWith(left: Expression, right: 
Expression)
   }
 }
 
+object StringTranslate {
+
+  def buildDict(matchingString: UTF8String, replaceString: UTF8String)
+: JMap[Character, Character] = {
+val matching = matchingString.toString()
+val replace = replaceString.toString()
+val dict = new HashMap[Character, Character]()
+var i = 0
+while (i < matching.length()) {
+  val rep = if (i < replace.length()) replace.charAt(i) else '\0'
+  if (null == dict.get(matching.charAt(i))) {
+dict.put(matching.charAt(i), rep)
+  }
+  i += 1
+}
+dict
+  }
+}
+
+/**
+ * A function translate any character in the `srcExpr` by a character in 
`replaceExpr`.
+ * The characters in `replaceExpr` is corresponding to the characters in 
`matchingExpr`.
+ * The translate will happen when any character in the string matching 
with the character
+ * in the `matchingExpr`.
+ */
+case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, 
replaceExpr: Expression)
+  extends TernaryExpression with ImplicitCastInputTypes {
+
+  @transient private var lastMatching: UTF8String = _
+  @transient private var lastReplace: UTF8String = _
+  @transient private var dict: JMap[Character, Character] = _
+
+  override def nullSafeEval(srcEval: Any, matchingEval: Any, replaceEval: 
Any): Any = {
+if (matchingEval != lastMatching || replaceEval != lastReplace) {
+  lastMatching = matchingEval.asInstanceOf[UTF8String]
--- End diff --

@davies just not quite sure why we need clone() here, seems like the 
UTF8String implementation is kind of like String which do the "copy on write". 
I guess we can somehow treat it as a constant value?   


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128244303
  
  [Test build #39992 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39992/consoleFull)
 for   PR 7709 at commit 
[`f2ab77a`](https://github.com/apache/spark/commit/f2ab77aceec9e0ec96b2e13e2a0c4ec1488c8f4f).


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

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



[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7954#issuecomment-128244284
  
  [Test build #39991 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39991/consoleFull)
 for   PR 7954 at commit 
[`34fa17b`](https://github.com/apache/spark/commit/34fa17bc9f1b85a0f3ae3bf1c2465c03990aa470).


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128244086
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7954#issuecomment-128244075
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9630] [SQL] Clean up new aggregate oper...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7954#issuecomment-128244059
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7709#issuecomment-128244061
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381800
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
 ---
@@ -241,14 +245,37 @@ public void setFloat(int ordinal, float value) {
   @Override
   public void setDecimal(int ordinal, Decimal value, int precision) {
--- End diff --

can you update InternalRow and UnsafeRow's documentation to say in order to 
set null on a decimal field, the caller must use setDecimal(ordinal, null, ...) 
instead of setNull?


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381720
  
--- Diff: 
unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java ---
@@ -107,6 +116,10 @@ public static void freeMemory(long address) {
 
   public static final int DOUBLE_ARRAY_OFFSET;
 
+  // Support for resetting final fields while deserializing
+  public static final long BigIntegerSignumOffset;
--- End diff --

BIG_INTEGER_SIGNUM_OFFSET and BIG_INTEGER_MAG_OFFSET


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381700
  
--- Diff: 
unsafe/src/main/java/org/apache/spark/unsafe/PlatformDependent.java ---
@@ -129,11 +142,26 @@ public static void freeMemory(long address) {
   INT_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(int[].class);
   LONG_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(long[].class);
   DOUBLE_ARRAY_OFFSET = _UNSAFE.arrayBaseOffset(double[].class);
+
+  long signumOffset = 0;
+  long magOffset = 0;
+  try {
+signumOffset = unsafe.objectFieldOffset
+  (BigInteger.class.getDeclaredField("signum"));
--- End diff --

why wrapping here and next line?


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

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



[GitHub] spark pull request: [CORE] Remove space in function for scala styl...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7984#issuecomment-128243801
  
Merged build finished. Test PASSed.


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

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



[GitHub] spark pull request: [CORE] Remove space in function for scala styl...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7984#issuecomment-128243766
  
  [Test build #39976 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39976/console)
 for   PR 7984 at commit 
[`43d1709`](https://github.com/apache/spark/commit/43d1709d5ca9173a66f77a0d9db356f021e5328c).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381662
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
 ---
@@ -87,12 +87,16 @@ public static int calculateBitSetWidthInBytes(int 
numFields) {
 
   public static boolean isFixedLength(DataType dt) {
 if (dt instanceof DecimalType) {
-  return ((DecimalType) dt).precision() < Decimal.MAX_LONG_DIGITS();
+  return ((DecimalType) dt).precision() <= Decimal.MAX_LONG_DIGITS();
 } else {
   return settableFieldTypes.contains(dt);
 }
   }
 
+  public static boolean isSettable(DataType dt) {
--- End diff --

ah ic you can't combine them since it's used in other places as well -- in 
that case renaming this to isMutable makes more sense.




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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381611
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
 ---
@@ -87,12 +87,16 @@ public static int calculateBitSetWidthInBytes(int 
numFields) {
 
   public static boolean isFixedLength(DataType dt) {
 if (dt instanceof DecimalType) {
-  return ((DecimalType) dt).precision() < Decimal.MAX_LONG_DIGITS();
+  return ((DecimalType) dt).precision() <= Decimal.MAX_LONG_DIGITS();
 } else {
   return settableFieldTypes.contains(dt);
 }
   }
 
+  public static boolean isSettable(DataType dt) {
--- End diff --

isMutable is a better 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: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381608
  
--- Diff: 
sql/catalyst/src/main/java/org/apache/spark/sql/catalyst/expressions/UnsafeRow.java
 ---
@@ -87,12 +87,16 @@ public static int calculateBitSetWidthInBytes(int 
numFields) {
 
   public static boolean isFixedLength(DataType dt) {
 if (dt instanceof DecimalType) {
-  return ((DecimalType) dt).precision() < Decimal.MAX_LONG_DIGITS();
+  return ((DecimalType) dt).precision() <= Decimal.MAX_LONG_DIGITS();
 } else {
   return settableFieldTypes.contains(dt);
 }
   }
 
+  public static boolean isSettable(DataType dt) {
--- End diff --

how about just merging this into isFixedLength, or merging isFixedLength 
into isSettable?


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

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



[GitHub] spark pull request: [SPARK-9644] [SQL] Support update DecimalType ...

2015-08-05 Thread rxin
Github user rxin commented on a diff in the pull request:

https://github.com/apache/spark/pull/7978#discussion_r36381589
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/codegen/GenerateUnsafeProjection.scala
 ---
@@ -146,13 +146,24 @@ object GenerateUnsafeProjection extends 
CodeGenerator[Seq[Expression], UnsafePro
 
 val fieldWriters = inputTypes.zip(convertedFields).zipWithIndex.map { 
case ((dt, ev), i) =>
   val update = genFieldWriter(ctx, dt, ev, output, i, cursor)
-  s"""
+  if (dt.isInstanceOf[DecimalType]) {
+// Can't call setNullAt() for DecimalType
+s"""
+ if (${ev.isNull}) {
+   $cursor += $DecimalWriter.write($output, $i, $cursor, null);
+ } else {
+   $update;
+ }
+ """
+  } else {
+s"""
 if (${ev.isNull}) {
   $output.setNullAt($i);
 } else {
   $update;
 }
   """
--- End diff --

this is aligned really weirdly right now


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

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



[GitHub] spark pull request: [SQL] [WIP] Join test coverage improvements

2015-08-05 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/7985#discussion_r36381603
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastNestedLoopJoin.scala
 ---
@@ -65,8 +65,9 @@ case class BroadcastNestedLoopJoin(
 left.output.map(_.withNullability(true)) ++ right.output
   case FullOuter =>
 left.output.map(_.withNullability(true)) ++ 
right.output.map(_.withNullability(true))
-  case _ =>
-left.output ++ right.output
+  case x =>
+throw new IllegalArgumentException(
--- End diff --

BroadcastNestedLoopJoin can't handle non-outer joins, but there was nothing 
to guard against this (this would never end up being planned though).


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

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



[GitHub] spark pull request: [SPARK-9674][SPARK-9667] Remove SparkSqlSerial...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128242600
  
  [Test build #39990 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39990/consoleFull)
 for   PR 7981 at commit 
[`82d1dc7`](https://github.com/apache/spark/commit/82d1dc73a05d4b5605acac02f9f0ebce1ecdf6f0).


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

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



[GitHub] spark pull request: [SPARK-8266][SQL]add function translate

2015-08-05 Thread davies
Github user davies commented on a diff in the pull request:

https://github.com/apache/spark/pull/7709#discussion_r36381449
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/stringOperations.scala
 ---
@@ -349,6 +351,78 @@ case class EndsWith(left: Expression, right: 
Expression)
   }
 }
 
+object StringTranslate {
+
+  def buildDict(matchingString: UTF8String, replaceString: UTF8String)
+: JMap[Character, Character] = {
+val matching = matchingString.toString()
+val replace = replaceString.toString()
+val dict = new HashMap[Character, Character]()
+var i = 0
+while (i < matching.length()) {
+  val rep = if (i < replace.length()) replace.charAt(i) else '\0'
+  if (null == dict.get(matching.charAt(i))) {
+dict.put(matching.charAt(i), rep)
+  }
+  i += 1
+}
+dict
+  }
+}
+
+/**
+ * A function translate any character in the `srcExpr` by a character in 
`replaceExpr`.
+ * The characters in `replaceExpr` is corresponding to the characters in 
`matchingExpr`.
+ * The translate will happen when any character in the string matching 
with the character
+ * in the `matchingExpr`.
+ */
+case class StringTranslate(srcExpr: Expression, matchingExpr: Expression, 
replaceExpr: Expression)
+  extends TernaryExpression with ImplicitCastInputTypes {
+
+  @transient private var lastMatching: UTF8String = _
+  @transient private var lastReplace: UTF8String = _
+  @transient private var dict: JMap[Character, Character] = _
+
+  override def nullSafeEval(srcEval: Any, matchingEval: Any, replaceEval: 
Any): Any = {
+if (matchingEval != lastMatching || replaceEval != lastReplace) {
+  lastMatching = matchingEval.asInstanceOf[UTF8String]
--- End diff --

.clone()


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128240706
  
  [Test build #39989 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39989/consoleFull)
 for   PR 7977 at commit 
[`2197d37`](https://github.com/apache/spark/commit/2197d379d350e256a87f9647b778dcd9509e1f75).


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

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



[GitHub] spark pull request: [SPARK-9674][SPARK-9667] Remove SparkSqlSerial...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128240368
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128240434
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9674][SPARK-9667] Remove SparkSqlSerial...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128240429
  
Merged build started.


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

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



[GitHub] spark pull request: [SPARK-9548][SQL] Add a destructive iterator f...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/7924#issuecomment-128240607
  
  [Test build #39988 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39988/consoleFull)
 for   PR 7924 at commit 
[`7652083`](https://github.com/apache/spark/commit/76520839f0702a78363631e35502a293802be6ab).


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

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



[GitHub] spark pull request: [SPARK-9659][SQL] Rename inSet to isin to matc...

2015-08-05 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7977#issuecomment-128240388
  
 Merged build triggered.


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

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



[GitHub] spark pull request: [SPARK-7165] [SQL] use sort merge join for out...

2015-08-05 Thread SparkQA
Github user SparkQA commented on the pull request:

https://github.com/apache/spark/pull/5717#issuecomment-128239391
  
  [Test build #39983 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/39983/console)
 for   PR 5717 at commit 
[`d02f6bb`](https://github.com/apache/spark/commit/d02f6bbab969a20e8a3cd9d6b065db39462d6ff5).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



  1   2   3   4   5   6   7   8   9   10   >