[GitHub] spark pull request: [SPARK-8345] [ML] Add an SQL node as a feature...

2015-08-06 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/7465#discussion_r36495071
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala ---
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.feature
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.ml.param.{ParamMap, Param}
+import org.apache.spark.ml.Transformer
+import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.sql.{SQLContext, DataFrame, Row}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * :: Experimental ::
+ * Implements the transforms which are defined by SQL statement.
+ * Currently we only support SQL syntax like 'SELECT ... FROM __THIS__'
+ * where '__THIS__' represents the underlying table of the input dataset.
+ */
+@Experimental
+class SQLTransformer (override val uid: String) extends Transformer {
+
+  def this() = this(Identifiable.randomUID("sql"))
+
+  /**
+   * SQL statement parameter. The statement is provided in string form.
+   * @group param
+   */
+  final val statement: Param[String] = new Param[String](this, 
"statement", "SQL statement")
+
+  /** @group setParam */
+  def setStatement(value: String): this.type = set(statement, value)
+
+  /** @group getParam */
+  def getStatement(): String = $(statement)
+
+  private val tableIdentifier: String = "__THIS__"
+
+  override def transform(dataset: DataFrame): DataFrame = {
+val outputSchema = transformSchema(dataset.schema, logging = true)
+val tableName = Identifiable.randomUID("sql")
--- End diff --

Use the `uid` of this transformer instance with a random suffix.


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

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

https://github.com/apache/spark/pull/7904#discussion_r36495087
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/ShuffledHashOuterJoin.scala
 ---
@@ -1,97 +1,94 @@
-/*
- * 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.joins
-
-import scala.collection.JavaConversions._
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.plans.{FullOuter, JoinType, 
LeftOuter, RightOuter}
-import org.apache.spark.sql.execution.{BinaryNode, SparkPlan}
-
-/**
- * :: DeveloperApi ::
- * Performs a hash based outer join for two child relations by shuffling 
the data using
- * the join keys. This operator requires loading the associated partition 
in both side into memory.
- */
-@DeveloperApi
-case class ShuffledHashOuterJoin(
-leftKeys: Seq[Expression],
-rightKeys: Seq[Expression],
-joinType: JoinType,
-condition: Option[Expression],
-left: SparkPlan,
-right: SparkPlan) extends BinaryNode with HashOuterJoin {
-
-  override def requiredChildDistribution: Seq[Distribution] =
-ClusteredDistribution(leftKeys) :: ClusteredDistribution(rightKeys) :: 
Nil
-
-  override def outputPartitioning: Partitioning = joinType match {
-case LeftOuter => left.outputPartitioning
-case RightOuter => right.outputPartitioning
-case FullOuter => 
UnknownPartitioning(left.outputPartitioning.numPartitions)
-case x =>
-  throw new IllegalArgumentException(s"HashOuterJoin should not take 
$x as the JoinType")
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-val joinedRow = new JoinedRow()
-left.execute().zipPartitions(right.execute()) { (leftIter, rightIter) 
=>
-  // TODO this probably can be replaced by external sort (sort merged 
join?)
-  joinType match {
-case LeftOuter =>
-  val hashed = HashedRelation(rightIter, buildKeyGenerator)
-  val keyGenerator = streamedKeyGenerator
-  val resultProj = resultProjection
-  leftIter.flatMap( currentRow => {
-val rowKey = keyGenerator(currentRow)
-joinedRow.withLeft(currentRow)
-leftOuterIterator(rowKey, joinedRow, hashed.get(rowKey), 
resultProj)
-  })
-
-case RightOuter =>
-  val hashed = HashedRelation(leftIter, buildKeyGenerator)
-  val keyGenerator = streamedKeyGenerator
-  val resultProj = resultProjection
-  rightIter.flatMap ( currentRow => {
-val rowKey = keyGenerator(currentRow)
-joinedRow.withRight(currentRow)
-rightOuterIterator(rowKey, hashed.get(rowKey), joinedRow, 
resultProj)
-  })
-
-case FullOuter =>
-  // TODO(davies): use UnsafeRow
-  val leftHashTable = buildHashTable(leftIter, 
newProjection(leftKeys, left.output))
-  val rightHashTable = buildHashTable(rightIter, 
newProjection(rightKeys, right.output))
-  (leftHashTable.keySet ++ rightHashTable.keySet).iterator.flatMap 
{ key =>
-fullOuterIterator(key,
-  leftHashTable.getOrElse(key, EMPTY_LIST),
-  rightHashTable.getOrElse(key, EMPTY_LIST),
-  joinedRow)
-  }
-
-case x =>
-  throw new IllegalArgumentException(
-s"ShuffledHashOuterJoin should not take $x as the JoinType")
-  }
-}
-  }
-}
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additiona

[GitHub] spark pull request: [SPARK-9683][SQL] copy UTF8String when convert...

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

https://github.com/apache/spark/pull/7990#issuecomment-128616625
  
LGTM


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

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



[GitHub] spark pull request: [SPARK-9683][SQL] copy UTF8String when convert...

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7990#discussion_r36495089
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FromUnsafe.scala
 ---
@@ -52,6 +53,8 @@ case class FromUnsafe(child: Expression) extends 
UnaryExpression
   }
   new GenericArrayData(result)
 
+case StringType => value.asInstanceOf[UTF8String].clone()
--- End diff --

Yes, I know, I assume we need to handle the top data type in `StringType` 
also, otherwise, it still will cause confusing if people call the `FromUnsafe` 
to get the `StringType` of data.

I mean we'd better add the `StringType` support in 
https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala#L159


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

2015-08-06 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/7465#discussion_r36495070
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala ---
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.feature
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.ml.param.{ParamMap, Param}
+import org.apache.spark.ml.Transformer
+import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.sql.{SQLContext, DataFrame, Row}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * :: Experimental ::
+ * Implements the transforms which are defined by SQL statement.
+ * Currently we only support SQL syntax like 'SELECT ... FROM __THIS__'
+ * where '__THIS__' represents the underlying table of the input dataset.
+ */
+@Experimental
+class SQLTransformer (override val uid: String) extends Transformer {
+
+  def this() = this(Identifiable.randomUID("sql"))
+
+  /**
+   * SQL statement parameter. The statement is provided in string form.
+   * @group param
+   */
+  final val statement: Param[String] = new Param[String](this, 
"statement", "SQL statement")
+
+  /** @group setParam */
+  def setStatement(value: String): this.type = set(statement, value)
+
+  /** @group getParam */
+  def getStatement(): String = $(statement)
+
+  private val tableIdentifier: String = "__THIS__"
+
+  override def transform(dataset: DataFrame): DataFrame = {
+val outputSchema = transformSchema(dataset.schema, logging = true)
+val tableName = Identifiable.randomUID("sql")
+dataset.registerTempTable(tableName)
+val realStatement = $(statement).replace(tableIdentifier, tableName)
+val additiveDF = dataset.sqlContext.sql(realStatement)
+val rdd = dataset.rdd.zip(additiveDF.rdd).map {
--- End diff --

I think it is okay to return the DataFrame from `sqlContext.sql` directly. 
User should use `*` if they want to keep existing columns.


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

2015-08-06 Thread mengxr
Github user mengxr commented on a diff in the pull request:

https://github.com/apache/spark/pull/7465#discussion_r36495073
  
--- Diff: 
mllib/src/main/scala/org/apache/spark/ml/feature/SQLTransformer.scala ---
@@ -0,0 +1,81 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.ml.feature
+
+import org.apache.spark.SparkContext
+import org.apache.spark.annotation.Experimental
+import org.apache.spark.ml.param.{ParamMap, Param}
+import org.apache.spark.ml.Transformer
+import org.apache.spark.ml.util.Identifiable
+import org.apache.spark.sql.{SQLContext, DataFrame, Row}
+import org.apache.spark.sql.types.StructType
+
+/**
+ * :: Experimental ::
+ * Implements the transforms which are defined by SQL statement.
+ * Currently we only support SQL syntax like 'SELECT ... FROM __THIS__'
+ * where '__THIS__' represents the underlying table of the input dataset.
+ */
+@Experimental
+class SQLTransformer (override val uid: String) extends Transformer {
+
+  def this() = this(Identifiable.randomUID("sql"))
+
+  /**
+   * SQL statement parameter. The statement is provided in string form.
+   * @group param
+   */
+  final val statement: Param[String] = new Param[String](this, 
"statement", "SQL statement")
+
+  /** @group setParam */
+  def setStatement(value: String): this.type = set(statement, value)
+
+  /** @group getParam */
+  def getStatement(): String = $(statement)
--- End diff --

Remove `()`


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

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

https://github.com/apache/spark/pull/8026#issuecomment-128616385
  
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-9683][SQL] copy UTF8String when convert...

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

https://github.com/apache/spark/pull/7990#issuecomment-128616462
  
  [Test build #40153 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40153/consoleFull)
 for   PR 7990 at commit 
[`c13d1e3`](https://github.com/apache/spark/commit/c13d1e362109d16cea2d6181f6dbf24c9097a7c3).


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

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

https://github.com/apache/spark/pull/8026#issuecomment-128616376
  
 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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#discussion_r36495029
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/BroadcastHashOuterJoin.scala
 ---
@@ -1,122 +1,119 @@
-/*
- * 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.joins
-
-import scala.concurrent._
-import scala.concurrent.duration._
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.physical.{Distribution, 
Partitioning, UnspecifiedDistribution}
-import org.apache.spark.sql.catalyst.plans.{JoinType, LeftOuter, 
RightOuter}
-import org.apache.spark.sql.execution.{BinaryNode, SQLExecution, SparkPlan}
-import org.apache.spark.{InternalAccumulator, TaskContext}
-
-/**
- * :: DeveloperApi ::
- * Performs a outer hash join for two child relations.  When the output 
RDD of this operator is
- * being constructed, a Spark job is asynchronously started to calculate 
the values for the
- * broadcasted relation.  This data is then placed in a Spark broadcast 
variable.  The streamed
- * relation is not shuffled.
- */
-@DeveloperApi
-case class BroadcastHashOuterJoin(
-leftKeys: Seq[Expression],
-rightKeys: Seq[Expression],
-joinType: JoinType,
-condition: Option[Expression],
-left: SparkPlan,
-right: SparkPlan) extends BinaryNode with HashOuterJoin {
-
-  val timeout = {
-val timeoutValue = sqlContext.conf.broadcastTimeout
-if (timeoutValue < 0) {
-  Duration.Inf
-} else {
-  timeoutValue.seconds
-}
-  }
-
-  override def requiredChildDistribution: Seq[Distribution] =
-UnspecifiedDistribution :: UnspecifiedDistribution :: Nil
-
-  override def outputPartitioning: Partitioning = 
streamedPlan.outputPartitioning
-
-  // Use lazy so that we won't do broadcast when calling explain but still 
cache the broadcast value
-  // for the same query.
-  @transient
-  private lazy val broadcastFuture = {
-// broadcastFuture is used in "doExecute". Therefore we can get the 
execution id correctly here.
-val executionId = 
sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
-future {
-  // This will run in another thread. Set the execution id so that we 
can connect these jobs
-  // with the correct execution.
-  SQLExecution.withExecutionId(sparkContext, executionId) {
-// Note that we use .execute().collect() because we don't want to 
convert data to Scala
-// types
-val input: Array[InternalRow] = 
buildPlan.execute().map(_.copy()).collect()
-val hashed = HashedRelation(input.iterator, buildKeyGenerator, 
input.size)
-sparkContext.broadcast(hashed)
-  }
-}(BroadcastHashJoin.broadcastHashJoinExecutionContext)
-  }
-
-  protected override def doPrepare(): Unit = {
-broadcastFuture
-  }
-
-  override def doExecute(): RDD[InternalRow] = {
-val broadcastRelation = Await.result(broadcastFuture, timeout)
-
-streamedPlan.execute().mapPartitions { streamedIter =>
-  val joinedRow = new JoinedRow()
-  val hashTable = broadcastRelation.value
-  val keyGenerator = streamedKeyGenerator
-
-  hashTable match {
-case unsafe: UnsafeHashedRelation =>
-  TaskContext.get().internalMetricsToAccumulators(
-
InternalAccumulator.PEAK_EXECUTION_MEMORY).add(unsafe.getUnsafeSize)
-case _ =>
-  }
-
-  val resultProj = resultProjection
-  joinType match {
-case LeftOuter =>
-  streamedIter.flatMap(currentRow => {
-val rowKey = keyGenerator(currentRow)
-joinedRow.withLeft(currentRow)
  

[GitHub] spark pull request: [SPARK-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128616394
  
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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128616378
  
 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-9735][SQL]Respect the user specified sc...

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/8026#issuecomment-128615201
  
cc @liancheng 


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

2015-08-06 Thread chenghao-intel
GitHub user chenghao-intel opened a pull request:

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

[SPARK-9735][SQL]Respect the user specified schema than the infer partition 
schema for HadoopFsRelation

To enable the unit test of `hadoopFsRelationSuite.Partition column type 
casting`. It previously threw exception like:
···
11.521 ERROR org.apache.spark.executor.Executor: Exception in task 2.0 in 
stage 2.0 (TID 130)
java.lang.ClassCastException: java.lang.Integer cannot be cast to 
org.apache.spark.unsafe.types.UTF8String
at 
org.apache.spark.sql.catalyst.expressions.BaseGenericInternalRow$class.getUTF8String(rows.scala:45)
at 
org.apache.spark.sql.catalyst.expressions.SpecificMutableRow.getUTF8String(SpecificMutableRow.scala:195)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toScalaImpl(CatalystTypeConverters.scala:297)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$StringConverter$.toScalaImpl(CatalystTypeConverters.scala:289)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$CatalystTypeConverter.toScala(CatalystTypeConverters.scala:110)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toScala(CatalystTypeConverters.scala:278)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$StructConverter.toScala(CatalystTypeConverters.scala:245)
at 
org.apache.spark.sql.catalyst.CatalystTypeConverters$$anonfun$createToScalaConverter$2.apply(CatalystTypeConverters.scala:406)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$3$$anonfun$apply$2.apply(SparkPlan.scala:194)
at 
org.apache.spark.sql.execution.SparkPlan$$anonfun$3$$anonfun$apply$2.apply(SparkPlan.scala:194)
at scala.collection.Iterator$$anon$11.next(Iterator.scala:328)
at scala.collection.Iterator$class.foreach(Iterator.scala:727)
at scala.collection.AbstractIterator.foreach(Iterator.scala:1157)
at 
scala.collection.generic.Growable$class.$plus$plus$eq(Growable.scala:48)
at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:103)
at 
scala.collection.mutable.ArrayBuffer.$plus$plus$eq(ArrayBuffer.scala:47)
at scala.collection.TraversableOnce$class.to(TraversableOnce.scala:273)
at scala.collection.AbstractIterator.to(Iterator.scala:1157)
at 
scala.collection.TraversableOnce$class.toBuffer(TraversableOnce.scala:265)
at scala.collection.AbstractIterator.toBuffer(Iterator.scala:1157)
at 
scala.collection.TraversableOnce$class.toArray(TraversableOnce.scala:252)
at scala.collection.AbstractIterator.toArray(Iterator.scala:1157)
at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:905)
at 
org.apache.spark.rdd.RDD$$anonfun$collect$1$$anonfun$12.apply(RDD.scala:905)
at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1836)
at 
org.apache.spark.SparkContext$$anonfun$runJob$5.apply(SparkContext.scala:1836)
at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
at org.apache.spark.scheduler.Task.run(Task.scala:88)
···

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

$ git pull https://github.com/chenghao-intel/spark partition_discovery

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

https://github.com/apache/spark/pull/8026.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #8026


commit 637e26fec1c00cad457f5ae92200b5f6700f1e36
Author: Cheng Hao 
Date:   2015-08-07T06:45:21Z

make lower priority of infer partition schema for HadoopFsRelation




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

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

https://github.com/apache/spark/pull/7990#issuecomment-128614684
  
 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-9683][SQL] copy UTF8String when convert...

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

https://github.com/apache/spark/pull/7990#issuecomment-128614705
  
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-9593] [SQL] [HOTFIX] Makes the Hadoop s...

2015-08-06 Thread liancheng
Github user liancheng commented on the pull request:

https://github.com/apache/spark/pull/7994#issuecomment-128614281
  
This issue seems to be fixed according to recent build results of the SBT 
master builder and my experimental builder. I've stopped the experimental 
builder, but didn't delete it, just in case we this issue comes out again (it 
has been behaving non-deterministically).


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

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

https://github.com/apache/spark/pull/7904#issuecomment-128613934
  
  [Test build #40151 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40151/console)
 for   PR 7904 at commit 
[`82b7e45`](https://github.com/apache/spark/commit/82b7e452964d10284a0639e2027abe750dd966c0).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request: [SPARK-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128613936
  
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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128613496
  
  [Test build #40151 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40151/consoleFull)
 for   PR 7904 at commit 
[`82b7e45`](https://github.com/apache/spark/commit/82b7e452964d10284a0639e2027abe750dd966c0).


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with 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-9726] [Python] PySpark DF join no longe...

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

https://github.com/apache/spark/pull/8016#issuecomment-128613469
  
LGTM, thanks to working on this!


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

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



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

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

https://github.com/apache/spark/pull/7981#issuecomment-128613446
  
  [Test build #40152 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40152/consoleFull)
 for   PR 7981 at commit 
[`58b7332`](https://github.com/apache/spark/commit/58b73323a5f4a166255b32b002267b3a4407d1a7).


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

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

https://github.com/apache/spark/pull/8005#issuecomment-128613097
  
Merging in master & branch-1.5.



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

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



[GitHub] spark pull request: [SPARK-9453] [SQL] support records larger than...

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

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


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

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



[GitHub] spark pull request: [SPARK-9453] [SQL] support records larger than...

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

https://github.com/apache/spark/pull/8005#issuecomment-128612932
  
  [Test build #40138 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40138/console)
 for   PR 8005 at commit 
[`f9c4aff`](https://github.com/apache/spark/commit/f9c4aff56c966ed002400c827461b83715c2fd9e).
 * 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-5155] [PySpark] [Streaming] Mqtt stream...

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

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


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

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



[GitHub] spark pull request: [SPARK-5155] [PySpark] [Streaming] Mqtt stream...

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

https://github.com/apache/spark/pull/7833#issuecomment-128612863
  
  [Test build #40141 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40141/console)
 for   PR 7833 at commit 
[`9570bec`](https://github.com/apache/spark/commit/9570bec0d54537e51623b2b5777895c209dd706a).
 * This patch **passes all tests**.
 * This patch merges cleanly.
 * This patch adds the following public classes _(experimental)_:
  * `class MQTTUtils(object):`



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with 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-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128612856
  
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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128612840
  
 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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128612857
  
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-06 Thread AmplabJenkins
Github user AmplabJenkins commented on the pull request:

https://github.com/apache/spark/pull/7981#issuecomment-128612833
  
 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-9683][SQL] we should deep copy UTF8Stri...

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

https://github.com/apache/spark/pull/7990#discussion_r36493967
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
 ---
@@ -87,4 +91,36 @@ class RowFormatConvertersSuite extends SparkPlanTest {
   input.map(Row.fromTuple)
 )
   }
+
+  test("SPARK-9683: we should deep copy UTF8String when convert unsafe row 
to safe row") {
+SparkPlan.currentContext.set(TestSQLContext)
+val schema = ArrayType(StringType)
+val rows = (1 to 100).map { i =>
+  InternalRow(new 
GenericArrayData(Array[Any](UTF8String.fromString(i.toString
+}
+val relation = LocalTableScan(Seq(AttributeReference("t", schema)()), 
rows)
+
+val plan =
+  DummyPlan(
+ConvertToSafe(
+  ConvertToUnsafe(relation)))
+assert(plan.execute().collect().map(_.getUTF8String(0).toString) === 
(1 to 100).map(_.toString))
+  }
+}
+
+case class DummyPlan(child: SparkPlan) extends UnaryNode {
+
+  override protected def doExecute(): RDD[InternalRow] = {
+child.execute().mapPartitions { iter =>
+  // cache all strings to make sure we have deep copied UTF8String 
inside incoming
+  // safe InternalRow.
+  val strings = new scala.collection.mutable.ArrayBuffer[UTF8String]
+  iter.foreach { row =>
+strings += row.getArray(0).getUTF8String(0)
--- End diff --

Could you add a comment here to say "This is working in safe mode, so we 
don't need to copy the UTF8String" ?


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

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

https://github.com/apache/spark/pull/7990#issuecomment-128611173
  
@cloud-fan The changes looks good to me, but the title does not reflect the 
problem here, it make me confusing in the begging. Could you change it to say 
"copy UTF8String when convert unsafe array/map to safe"?


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

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

https://github.com/apache/spark/pull/7802#issuecomment-128611100
  
  [Test build #1402 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1402/consoleFull)
 for   PR 7802 at commit 
[`e5e93b2`](https://github.com/apache/spark/commit/e5e93b2a31d387dc23226adec6ece7a8c8704f7f).


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

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/7788#issuecomment-128611025
  
Mark it as WIP, I will take more time in investigating that how to 
integrate with Hive UDAF.


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

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

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


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

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



[GitHub] spark pull request: [SPARK-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128610404
  
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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128610394
  
  [Test build #40150 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40150/console)
 for   PR 7904 at commit 
[`bdf513c`](https://github.com/apache/spark/commit/bdf513c037eb9e5c746b848da884991938926dd9).
 * This patch **fails Scala style tests**.
 * This patch merges cleanly.
 * This patch adds no public classes.


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

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



[GitHub] spark pull request: [SPARK-9486][SQL] Add data source aliasing for...

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

https://github.com/apache/spark/pull/7802#issuecomment-128610460
  
  [Test build #1398 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1398/console)
 for   PR 7802 at commit 
[`e5e93b2`](https://github.com/apache/spark/commit/e5e93b2a31d387dc23226adec6ece7a8c8704f7f).
 * This patch **fails Spark unit tests**.
 * This patch **does not merge cleanly**.
 * This patch adds the following public classes _(experimental)_:
  * `class BlockMatrix(DistributedMatrix):`
  * `case class In(value: Expression, list: Seq[Expression]) extends 
Predicate`
  * `case class InSet(child: Expression, hset: Set[Any]) extends 
UnaryExpression with Predicate `
  * `sys.error(s"Failed to load class for data source: 
$provider")`
  * `trait DataSourceRegister `



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

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

https://github.com/apache/spark/pull/8020#issuecomment-128610624
  
  [Test build #40140 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40140/console)
 for   PR 8020 at commit 
[`0e15b12`](https://github.com/apache/spark/commit/0e15b1292e92a989f629a3e70e59f0e0e5e92d88).
 * 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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128609832
  
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-9729] [SPARK-9363] [SQL] Use sort merge...

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

https://github.com/apache/spark/pull/7904#issuecomment-128609509
  
  [Test build #40150 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40150/consoleFull)
 for   PR 7904 at commit 
[`bdf513c`](https://github.com/apache/spark/commit/bdf513c037eb9e5c746b848da884991938926dd9).


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

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

https://github.com/apache/spark/pull/7990#discussion_r36493512
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
 ---
@@ -87,4 +91,36 @@ class RowFormatConvertersSuite extends SparkPlanTest {
   input.map(Row.fromTuple)
 )
   }
+
+  test("SPARK-9683: we should deep copy UTF8String when convert unsafe row 
to safe row") {
+SparkPlan.currentContext.set(TestSQLContext)
+val schema = ArrayType(StringType)
+val rows = (1 to 100).map { i =>
+  InternalRow(new 
GenericArrayData(Array[Any](UTF8String.fromString(i.toString
+}
+val relation = LocalTableScan(Seq(AttributeReference("t", schema)()), 
rows)
+
+val plan =
+  DummyPlan(
+ConvertToSafe(
+  ConvertToUnsafe(relation)))
+assert(plan.execute().collect().map(_.getUTF8String(0).toString) === 
(1 to 100).map(_.toString))
+  }
+}
+
+case class DummyPlan(child: SparkPlan) extends UnaryNode {
+
+  override protected def doExecute(): RDD[InternalRow] = {
+child.execute().mapPartitions { iter =>
+  // cache all strings to make sure we have deep copied UTF8String 
inside incoming
+  // safe InternalRow.
+  val strings = new scala.collection.mutable.ArrayBuffer[UTF8String]
+  iter.foreach { row =>
+strings += row.getArray(0).getUTF8String(0)
--- End diff --

@cloud-fan Before we can make sure everything can work with Unsafe, please 
don't remove ConvertToSafe.


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

2015-08-06 Thread asfgit
Github user asfgit closed the pull request at:

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


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

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

https://github.com/apache/spark/pull/8012#issuecomment-128609377
  
Going to merge this.


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

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



[GitHub] spark pull request: [SPARK-9700] Pick default page size more intel...

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

https://github.com/apache/spark/pull/8012#issuecomment-128609246
  
  [Test build #1399 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/NewSparkPullRequestBuilder/1399/console)
 for   PR 8012 at commit 
[`16f4756`](https://github.com/apache/spark/commit/16f4756dad128a4474c6f59ec91aec9c54c114eb).
 * 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-9683][SQL] we should deep copy UTF8Stri...

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

https://github.com/apache/spark/pull/7990#discussion_r36493352
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FromUnsafe.scala
 ---
@@ -52,6 +53,8 @@ case class FromUnsafe(child: Expression) extends 
UnaryExpression
   }
   new GenericArrayData(result)
 
+case StringType => value.asInstanceOf[UTF8String].clone()
--- End diff --

For top level `UTF8String` in `UnsafeRow` field, yes, won't execute.
But for inner `UTF8String` inside array/map/struct, will execute, as we 
call this `convert` method recursively.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with 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-06 Thread JoshRosen
Github user JoshRosen commented on a diff in the pull request:

https://github.com/apache/spark/pull/5717#discussion_r36493358
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
 ---
@@ -62,111 +100,250 @@ case class SortMergeJoin(
   }
 
   protected override def doExecute(): RDD[InternalRow] = {
-val leftResults = left.execute().map(_.copy())
-val rightResults = right.execute().map(_.copy())
+val streamResults = streamedPlan.execute().map(_.copy())
+val bufferResults = bufferedPlan.execute().map(_.copy())
 
-leftResults.zipPartitions(rightResults) { (leftIter, rightIter) =>
+streamResults.zipPartitions(bufferResults) ( (streamedIter, 
bufferedIter) => {
+  // standard null rows
+  val streamedNullRow = 
InternalRow.fromSeq(Seq.fill(streamedPlan.output.length)(null))
+  val bufferedNullRow = 
InternalRow.fromSeq(Seq.fill(bufferedPlan.output.length)(null))
   new Iterator[InternalRow] {
 // An ordering that can be used to compare keys from both sides.
 private[this] val keyOrdering = 
newNaturalAscendingOrdering(leftKeys.map(_.dataType))
 // Mutable per row objects.
 private[this] val joinRow = new JoinedRow
-private[this] var leftElement: InternalRow = _
-private[this] var rightElement: InternalRow = _
-private[this] var leftKey: InternalRow = _
-private[this] var rightKey: InternalRow = _
-private[this] var rightMatches: CompactBuffer[InternalRow] = _
-private[this] var rightPosition: Int = -1
+private[this] var streamedElement: InternalRow = _
+private[this] var bufferedElement: InternalRow = _
+private[this] var streamedKey: InternalRow = _
+private[this] var bufferedKey: InternalRow = _
+private[this] var bufferedMatches: CompactBuffer[InternalRow] = _
+private[this] var bufferedPosition: Int = -1
 private[this] var stop: Boolean = false
 private[this] var matchKey: InternalRow = _
+// when we do merge algorithm and find some not matched join key, 
there must be a side
+// that do not have a corresponding match. So we need to mark 
which side it is. True means
+// streamed side not have match, and False means the buffered 
side. Only set when needed.
+private[this] var continueStreamed: Boolean = _
+private[this] var streamNullGenerated: Boolean = false
+// Tracks if each element in bufferedMatches have a matched 
streamedElement.
+private[this] var bitSet: BitSet = _
+// marks if the found result has been fetched.
+private[this] var found: Boolean = false
+private[this] var bufferNullGenerated: Boolean = false
 
 // initialize iterator
 initialize()
 
-override final def hasNext: Boolean = nextMatchingPair()
+override final def hasNext: Boolean = {
+  val matching = nextMatchingBlock()
+  if (matching && !isBufferEmpty(bufferedMatches)) {
+// The buffer stores all rows that match key, but condition 
may not be matched.
+// If none of rows in the buffer match condition, we'll fetch 
next matching block.
+findNextInBuffer() || hasNext
+  } else {
+matching
+  }
+}
+
+/**
+ * Run down the current `bufferedMatches` to find rows that match 
conditions.
+ * If `joinType` is not `Inner`, we will use `bufferNullGenerated` 
to mark if
+ * we need to build a bufferedNullRow for result.
+ * If `joinType` is `FullOuter`, we will use `streamNullGenerated` 
to mark if
+ * a buffered element need to join with a streamedNullRow.
+ * The method can be called multiple times since `found` serves as 
a guardian.
+ */
+def findNextInBuffer(): Boolean = {
+  while (!found && streamedElement != null
+&& keyOrdering.compare(streamedKey, matchKey) == 0) {
+while (bufferedPosition < bufferedMatches.size && 
!boundCondition(
+  joinRow(streamedElement, 
bufferedMatches(bufferedPosition {
+  bufferedPosition += 1
+}
+if (bufferedPosition == bufferedMatches.size) {
+  if (joinType == Inner || bufferNullGenerated) {
+bufferNullGenerated = false
+bufferedPosition = 0
+fetchStreamed()
+  } else {
+found = true
+  }
+} else {
+  // mark as true so we don't generate null row for streamed 
row.
+   

[GitHub] spark pull request: [SPARK-9729] [SPARK-9363] [WIP] [SQL] Use sort...

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

https://github.com/apache/spark/pull/7904#issuecomment-128608757
  
 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-9729] [SPARK-9363] [WIP] [SQL] Use sort...

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

https://github.com/apache/spark/pull/7904#issuecomment-128608772
  
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-9683][SQL] we should deep copy UTF8Stri...

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7990#discussion_r36493288
  
--- Diff: 
sql/core/src/test/scala/org/apache/spark/sql/execution/RowFormatConvertersSuite.scala
 ---
@@ -87,4 +91,36 @@ class RowFormatConvertersSuite extends SparkPlanTest {
   input.map(Row.fromTuple)
 )
   }
+
+  test("SPARK-9683: we should deep copy UTF8String when convert unsafe row 
to safe row") {
+SparkPlan.currentContext.set(TestSQLContext)
+val schema = ArrayType(StringType)
+val rows = (1 to 100).map { i =>
+  InternalRow(new 
GenericArrayData(Array[Any](UTF8String.fromString(i.toString
+}
+val relation = LocalTableScan(Seq(AttributeReference("t", schema)()), 
rows)
+
+val plan =
+  DummyPlan(
+ConvertToSafe(
+  ConvertToUnsafe(relation)))
+assert(plan.execute().collect().map(_.getUTF8String(0).toString) === 
(1 to 100).map(_.toString))
+  }
+}
+
+case class DummyPlan(child: SparkPlan) extends UnaryNode {
+
+  override protected def doExecute(): RDD[InternalRow] = {
+child.execute().mapPartitions { iter =>
+  // cache all strings to make sure we have deep copied UTF8String 
inside incoming
+  // safe InternalRow.
+  val strings = new scala.collection.mutable.ArrayBuffer[UTF8String]
+  iter.foreach { row =>
+strings += row.getArray(0).getUTF8String(0)
--- End diff --

I am wondering if this is the real case. As join / aggregation operations 
will call row.copy() if necessary.


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

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

https://github.com/apache/spark/pull/8017#issuecomment-128608097
  
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-9683][SQL] we should deep copy UTF8Stri...

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/7990#discussion_r36493225
  
--- Diff: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/FromUnsafe.scala
 ---
@@ -52,6 +53,8 @@ case class FromUnsafe(child: Expression) extends 
UnaryExpression
   }
   new GenericArrayData(result)
 
+case StringType => value.asInstanceOf[UTF8String].clone()
--- End diff --

This line probably never be executed.

https://github.com/apache/spark/blob/master/sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/expressions/Projection.scala#L159



---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with 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-06 Thread viirya
Github user viirya commented on a diff in the pull request:

https://github.com/apache/spark/pull/5717#discussion_r36493188
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/joins/SortMergeJoin.scala
 ---
@@ -62,111 +100,250 @@ case class SortMergeJoin(
   }
 
   protected override def doExecute(): RDD[InternalRow] = {
-val leftResults = left.execute().map(_.copy())
-val rightResults = right.execute().map(_.copy())
+val streamResults = streamedPlan.execute().map(_.copy())
+val bufferResults = bufferedPlan.execute().map(_.copy())
 
-leftResults.zipPartitions(rightResults) { (leftIter, rightIter) =>
+streamResults.zipPartitions(bufferResults) ( (streamedIter, 
bufferedIter) => {
+  // standard null rows
+  val streamedNullRow = 
InternalRow.fromSeq(Seq.fill(streamedPlan.output.length)(null))
+  val bufferedNullRow = 
InternalRow.fromSeq(Seq.fill(bufferedPlan.output.length)(null))
   new Iterator[InternalRow] {
 // An ordering that can be used to compare keys from both sides.
 private[this] val keyOrdering = 
newNaturalAscendingOrdering(leftKeys.map(_.dataType))
 // Mutable per row objects.
 private[this] val joinRow = new JoinedRow
-private[this] var leftElement: InternalRow = _
-private[this] var rightElement: InternalRow = _
-private[this] var leftKey: InternalRow = _
-private[this] var rightKey: InternalRow = _
-private[this] var rightMatches: CompactBuffer[InternalRow] = _
-private[this] var rightPosition: Int = -1
+private[this] var streamedElement: InternalRow = _
+private[this] var bufferedElement: InternalRow = _
+private[this] var streamedKey: InternalRow = _
+private[this] var bufferedKey: InternalRow = _
+private[this] var bufferedMatches: CompactBuffer[InternalRow] = _
+private[this] var bufferedPosition: Int = -1
 private[this] var stop: Boolean = false
 private[this] var matchKey: InternalRow = _
+// when we do merge algorithm and find some not matched join key, 
there must be a side
+// that do not have a corresponding match. So we need to mark 
which side it is. True means
+// streamed side not have match, and False means the buffered 
side. Only set when needed.
+private[this] var continueStreamed: Boolean = _
+private[this] var streamNullGenerated: Boolean = false
+// Tracks if each element in bufferedMatches have a matched 
streamedElement.
+private[this] var bitSet: BitSet = _
+// marks if the found result has been fetched.
+private[this] var found: Boolean = false
+private[this] var bufferNullGenerated: Boolean = false
 
 // initialize iterator
 initialize()
 
-override final def hasNext: Boolean = nextMatchingPair()
+override final def hasNext: Boolean = {
+  val matching = nextMatchingBlock()
+  if (matching && !isBufferEmpty(bufferedMatches)) {
+// The buffer stores all rows that match key, but condition 
may not be matched.
+// If none of rows in the buffer match condition, we'll fetch 
next matching block.
+findNextInBuffer() || hasNext
+  } else {
+matching
+  }
+}
+
+/**
+ * Run down the current `bufferedMatches` to find rows that match 
conditions.
+ * If `joinType` is not `Inner`, we will use `bufferNullGenerated` 
to mark if
+ * we need to build a bufferedNullRow for result.
+ * If `joinType` is `FullOuter`, we will use `streamNullGenerated` 
to mark if
+ * a buffered element need to join with a streamedNullRow.
+ * The method can be called multiple times since `found` serves as 
a guardian.
+ */
+def findNextInBuffer(): Boolean = {
+  while (!found && streamedElement != null
+&& keyOrdering.compare(streamedKey, matchKey) == 0) {
+while (bufferedPosition < bufferedMatches.size && 
!boundCondition(
+  joinRow(streamedElement, 
bufferedMatches(bufferedPosition {
+  bufferedPosition += 1
+}
+if (bufferedPosition == bufferedMatches.size) {
+  if (joinType == Inner || bufferNullGenerated) {
+bufferNullGenerated = false
+bufferedPosition = 0
+fetchStreamed()
+  } else {
+found = true
+  }
+} else {
+  // mark as true so we don't generate null row for streamed 
row.
+  

[GitHub] spark pull request: [SPARK-8890][SQL] Fallback on sorting when wri...

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

https://github.com/apache/spark/pull/8010#discussion_r36493200
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InsertIntoDataSource.scala
 ---
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.execution.datasources
+
+import java.io.IOException
+import java.util.{Date, UUID}
+
+import scala.collection.JavaConversions.asScalaIterator
+
+import org.apache.hadoop.fs.Path
+import org.apache.hadoop.mapreduce._
+import org.apache.hadoop.mapreduce.lib.output.{FileOutputCommitter => 
MapReduceFileOutputCommitter, FileOutputFormat}
+import org.apache.spark._
+import org.apache.spark.mapred.SparkHadoopMapRedUtil
+import org.apache.spark.mapreduce.SparkHadoopMapReduceUtil
+import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.codegen.GenerateProjection
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, Project}
+import org.apache.spark.sql.catalyst.{CatalystTypeConverters, InternalRow}
+import org.apache.spark.sql.execution.{RunnableCommand, SQLExecution}
+import org.apache.spark.sql.sources._
+import org.apache.spark.sql.types.StringType
+import org.apache.spark.util.{Utils, SerializableConfiguration}
+
+
+/**
+ * Inserts the results of `query` in to a relation that extends 
[[InsertableRelation]].
+ */
+private[sql] case class InsertIntoDataSource(
+logicalRelation: LogicalRelation,
+query: LogicalPlan,
+overwrite: Boolean)
+  extends RunnableCommand {
+
+  override def run(sqlContext: SQLContext): Seq[Row] = {
+val relation = 
logicalRelation.relation.asInstanceOf[InsertableRelation]
+val data = DataFrame(sqlContext, query)
+// Apply the schema of the existing table to the new data.
+val df = sqlContext.internalCreateDataFrame(data.queryExecution.toRdd, 
logicalRelation.schema)
+relation.insert(df, overwrite)
+
+// Invalidate the cache.
+sqlContext.cacheManager.invalidateCache(logicalRelation)
+
+Seq.empty[Row]
--- End diff --

Looks like the `InsertIntoDataSource` is inconsistent with 
`InsertIntoHadoopFsRelation`.

* In `InsertIntoHadoopFsRelation`, we do the insertion inside 
`SQLExecution.withNewExecutionId`, but here we don't.
* Here we invalidate the cache after insertion, but 
`InsertIntoHadoopFsRelation` don't.

Is this by intentional?


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

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

https://github.com/apache/spark/pull/7904#issuecomment-128608025
  
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-9729] [SPARK-9363] [WIP] [SQL] Use sort...

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

https://github.com/apache/spark/pull/7904#issuecomment-128608016
  
 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-8382][SQL] Improve Analysis Unit test f...

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

https://github.com/apache/spark/pull/8025#issuecomment-128607786
  
  [Test build #40148 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40148/consoleFull)
 for   PR 8025 at commit 
[`ec88ace`](https://github.com/apache/spark/commit/ec88ace1a9b85aea3dc458787c5596845f9839ed).


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

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

https://github.com/apache/spark/pull/8023#issuecomment-128607623
  
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-8382][SQL] Improve Analysis Unit test f...

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

https://github.com/apache/spark/pull/8025#issuecomment-128607566
  
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-8382][SQL] Improve Analysis Unit test f...

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

https://github.com/apache/spark/pull/8025#issuecomment-128607523
  
 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-8382][SQL] Improve Analysis Unit test f...

2015-08-06 Thread cloud-fan
GitHub user cloud-fan opened a pull request:

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

[SPARK-8382][SQL] Improve Analysis Unit test framework



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

$ git pull https://github.com/cloud-fan/spark analysis

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

https://github.com/apache/spark/pull/8025.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #8025


commit ec88ace1a9b85aea3dc458787c5596845f9839ed
Author: Wenchen Fan 
Date:   2015-08-07T06:00:55Z

Improve Analysis Unit test framework




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

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

https://github.com/apache/spark/pull/8018#issuecomment-128607278
  
  [Test build #40147 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40147/consoleFull)
 for   PR 8018 at commit 
[`c861001`](https://github.com/apache/spark/commit/c8610016a1581211d4a70c6b66be60df7104ac45).


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

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

https://github.com/apache/spark/pull/8018#issuecomment-128607011
  
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-9303] Decimal should use java.math.Deci...

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

https://github.com/apache/spark/pull/8018#issuecomment-128606998
  
 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-9689][SQL]Fix bug of not invalidate the...

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/8023#discussion_r36492704
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ---
@@ -88,7 +87,6 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
 
 // Scanning non-partitioned HadoopFsRelation
 case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
HadoopFsRelation)) =>
-  t.refresh()
--- End diff --

This is only called once when turn the logical plan to physical plan, 
however, the actual place we need is to refresh the files when physical plan -> 
RDD.


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

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/8023#discussion_r36492637
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ---
@@ -60,7 +60,6 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
 // Scanning partitioned HadoopFsRelation
 case PhysicalOperation(projects, filters, l @ LogicalRelation(t: 
HadoopFsRelation))
 if t.partitionSpec.partitionColumns.nonEmpty =>
-  t.refresh()
--- End diff --

This is an unnecessary change introduced by #7696


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

2015-08-06 Thread asfgit
Github user asfgit closed the pull request at:

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


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

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

https://github.com/apache/spark/pull/8017#issuecomment-128606407
  
 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-8862][SQL]Support multiple SQLContexts ...

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

https://github.com/apache/spark/pull/7962#issuecomment-128606243
  
I've merged this. Thanks.



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

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



[GitHub] spark pull request: [SPARK-9731] Standalone scheduling incorrect c...

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

https://github.com/apache/spark/pull/8017#issuecomment-128606422
  
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-9731] Standalone scheduling incorrect c...

2015-08-06 Thread carsonwang
Github user carsonwang commented on the pull request:

https://github.com/apache/spark/pull/8017#issuecomment-128606109
  
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-7550] [SQL] [MINOR] Fixes logs when per...

2015-08-06 Thread asfgit
Github user asfgit closed the pull request at:

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


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

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

https://github.com/apache/spark/pull/8024#issuecomment-128606081
  
  [Test build #40144 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40144/consoleFull)
 for   PR 8024 at commit 
[`5084675`](https://github.com/apache/spark/commit/5084675ce61e669c6eac8c19261b5195ed35bb2d).


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

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

https://github.com/apache/spark/pull/8021#issuecomment-128605958
  
Merging in master.



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

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



[GitHub] spark pull request: [SPARK-9689][SQL]Fix bug of not invalidate the...

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

https://github.com/apache/spark/pull/8023#issuecomment-128605635
  
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-9733][SQL] Improve explain message for ...

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

https://github.com/apache/spark/pull/8024#issuecomment-128605573
  
 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-9733][SQL] Improve explain message for ...

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

https://github.com/apache/spark/pull/8024#issuecomment-128605614
  
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-9689][SQL]Fix bug of not invalidate the...

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

https://github.com/apache/spark/pull/8023#issuecomment-128605586
  
 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-9303] Decimal should use java.math.Deci...

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

https://github.com/apache/spark/pull/8018#issuecomment-128605533
  
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-9303] Decimal should use java.math.Deci...

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

https://github.com/apache/spark/pull/8018#issuecomment-128605506
  
  [Test build #40139 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40139/console)
 for   PR 8018 at commit 
[`7b70c28`](https://github.com/apache/spark/commit/7b70c28a0043957f3ed25eaa62f14a085a45ae05).
 * 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-9733][SQL] Improve explain message for ...

2015-08-06 Thread rxin
GitHub user rxin opened a pull request:

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

[SPARK-9733][SQL] Improve explain message for data source scan node.

All data sources show up as "PhysicalRDD" in physical plan explain. It'd be 
better if we can show the name of the data source.

Without this patch:
```
== Physical Plan ==
NewAggregate with UnsafeHybridAggregationIterator ArrayBuffer(date#0, 
cat#1) ArrayBuffer((sum(CAST((CAST(count#2, IntegerType) + 1), 
LongType))2,mode=Final,isDistinct=false))
 Exchange hashpartitioning(date#0,cat#1)
  NewAggregate with UnsafeHybridAggregationIterator ArrayBuffer(date#0, 
cat#1) ArrayBuffer((sum(CAST((CAST(count#2, IntegerType) + 1), 
LongType))2,mode=Partial,isDistinct=false))
   PhysicalRDD [date#0,cat#1,count#2], MapPartitionsRDD[3] at
```

With this patch:
```
== Physical Plan ==
TungstenAggregate(key=[date#0,cat#1], value=[(sum(CAST((CAST(count#2, 
IntegerType) + 1), LongType)),mode=Final,isDistinct=false)]
 Exchange hashpartitioning(date#0,cat#1)
  TungstenAggregate(key=[date#0,cat#1], value=[(sum(CAST((CAST(count#2, 
IntegerType) + 1), LongType)),mode=Partial,isDistinct=false)]
   ConvertToUnsafe
Scan 
ParquetRelation[file:/scratch/rxin/spark/sales4][date#0,cat#1,count#2]
```

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

$ git pull https://github.com/rxin/spark SPARK-9733

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

https://github.com/apache/spark/pull/8024.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #8024


commit 5084675ce61e669c6eac8c19261b5195ed35bb2d
Author: Reynold Xin 
Date:   2015-08-07T05:45:13Z

[SPARK-9733][SQL] Improve explain message for data source scan node.




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

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on a diff in the pull request:

https://github.com/apache/spark/pull/8023#discussion_r36492462
  
--- Diff: 
sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/DataSourceStrategy.scala
 ---
@@ -98,8 +96,10 @@ private[sql] object DataSourceStrategy extends Strategy 
with Logging {
 l,
 projects,
 filters,
-(a, f) =>
-  toCatalystRDD(l, a, t.buildScan(a.map(_.name).toArray, f, 
t.paths, confBroadcast))) :: Nil
+(a, f) => {
+  t.refresh()
--- End diff --

Refresh the `HadoopFsRelation` right before making the rdd.


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

2015-08-06 Thread chenghao-intel
GitHub user chenghao-intel opened a pull request:

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

[SPARK-9689][SQL]Fix bug of not invalidate the cache for 
InsertIntoHadoopFsRelation

We didn't refresh the cache in `InsertIntoHadoopFsRelation`, however, even 
by adding the fresh operation, I also noticed that the spark plan is immutable, 
this is probably a bug if the underlying files changes (added/deleted).

So I mark the the PhyscialRdd as a mutable, and always create the new RDD 
while `doExecute()` function called.

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

$ git pull https://github.com/chenghao-intel/spark cache_refresh

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

https://github.com/apache/spark/pull/8023.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #8023


commit 0fcbda24efe84ea02163f47dd2d61b477f3de69f
Author: Cheng Hao 
Date:   2015-08-06T15:52:09Z

Fix bug of invalidate the cache for HadoopFsRelation




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

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/8023#issuecomment-128605307
  
cc @liancheng 


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

2015-08-06 Thread tdas
Github user tdas commented on the pull request:

https://github.com/apache/spark/pull/7961#issuecomment-128604965
  
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-6212][SQL]The EXPLAIN output of CTAS on...

2015-08-06 Thread chenghao-intel
Github user chenghao-intel commented on the pull request:

https://github.com/apache/spark/pull/7986#issuecomment-128604823
  
duplicated with #7980 :)


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

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

https://github.com/apache/spark/pull/8006#issuecomment-128604806
  
cc @JoshRosen for review


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

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



[GitHub] spark pull request: [SPARK-6902] [SQL] [PYSPARK] Row should be rea...

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

https://github.com/apache/spark/pull/8009#issuecomment-128604788
  
cc @JoshRosen for review


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

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



[GitHub] spark pull request: [SPARK-9674] Re-enable ignored test in SQLQuer...

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

https://github.com/apache/spark/pull/8015#issuecomment-128603507
  
  [Test build #40132 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40132/console)
 for   PR 8015 at commit 
[`225eac2`](https://github.com/apache/spark/commit/225eac229c34c1397d311da381c63b1942addee9).
 * 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-9674] Re-enable ignored test in SQLQuer...

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

https://github.com/apache/spark/pull/8015#issuecomment-128603714
  
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-9710] [test] Fix RPackageUtilsSuite whe...

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

https://github.com/apache/spark/pull/8008#issuecomment-128602302
  
  [Test build #40134 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40134/console)
 for   PR 8008 at commit 
[`df72d8c`](https://github.com/apache/spark/commit/df72d8cfc26f07f728ced55b885ec1ab197d3a5a).
 * 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-9710] [test] Fix RPackageUtilsSuite whe...

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

https://github.com/apache/spark/pull/8008#issuecomment-128602326
  
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-2017] [UI] Stage page hangs with many t...

2015-08-06 Thread pwendell
Github user pwendell commented on the pull request:

https://github.com/apache/spark/pull/7296#issuecomment-128600574
  
This is still good to have - it doesn't hurt to gzip the output


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with 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-9516][UI] Improvement of Thread Dump Pa...

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

https://github.com/apache/spark/pull/7910#issuecomment-128600321
  
Yeah, we've been combatting a really bad test flakiness issue over the past 
48 hours :(


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

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

https://github.com/apache/spark/pull/8022#issuecomment-128599819
  
Can one of the admins verify this patch?


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

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



[GitHub] spark pull request: [SPARK-9640][SPARK-9727][Streaming][Build][Tes...

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

https://github.com/apache/spark/pull/7961#issuecomment-128599380
  
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-9640][SPARK-9727][Streaming][Build][Tes...

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

https://github.com/apache/spark/pull/7961#issuecomment-128599311
  
**[Test build #40128 timed 
out](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40128/console)**
 for PR 7961 at commit 
[`ba9cfc7`](https://github.com/apache/spark/commit/ba9cfc72a108685e275ff3e7684add71792c1404)
 after a configured wait of `175m`.


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

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

https://github.com/apache/spark/pull/8014#issuecomment-128598367
  
  [Test build #40143 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/40143/consoleFull)
 for   PR 8014 at commit 
[`8a977cd`](https://github.com/apache/spark/commit/8a977cd3411c01a42e5b03eb2eaf244363bcb385).


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

2015-08-06 Thread rotationsymmetry
GitHub user rotationsymmetry opened a pull request:

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

[SPARK-4980] [MLlib] Add decay factors to streaming linear methods

This PR includes an implementation of decay factors in streaming linear and 
logistic regression. Unit tests are also included. 

The algorithm and design details are described in the document: 
https://docs.google.com/document/d/1UfKvuaaJVQCvh-wOLLYT8l7STQFjPxE7fitZyd0tqTo/edit?usp=sharing

Your comments and suggestions are highly appreciated. I will add more tests 
and ScalaDoc as suggested.

Thanks!


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

$ git pull https://github.com/rotationsymmetry/spark SPARK-4980-stream-decay

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

https://github.com/apache/spark/pull/8022.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #8022


commit 2248f11ee34ecb220ab7573b02513643083349e2
Author: Meihua Wu 
Date:   2015-07-24T06:56:48Z

Add decay to StreamingLinearAlgorithm through StreamingDecay trait.

Revise test "parameter accuracy" in StreamingLinearRegressionSuite to 
account for decay.

commit 12931aabbb455e1405a711465afe16c285cbd535
Author: Meihua Wu 
Date:   2015-07-25T05:06:00Z

Fix fluent setter API

Split StreamingDecay into two traits.
Update StreamingLogisticRegressionWithSGD.
Update test suites.

commit 0ab6ac07ad2d08368b147a34a699b391a635daa1
Author: Meihua Wu 
Date:   2015-08-07T04:46:13Z

Add unit tests.

Also make StreamingDecaySetter to be private[mllib].

commit 4bef2acaecfade7a8fccc051a7350abd8874e41e
Author: Meihua Wu 
Date:   2015-08-07T04:54:55Z

minor fixes




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