[GitHub] spark pull request: [SPARK-8345] [ML] Add an SQL node as a feature...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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 ...
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...
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 ...
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...
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...
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...
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 ...
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...
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...
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 ...
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 ...
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...
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...
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...
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 ...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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...
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