[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287551689
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
 ##
 @@ -0,0 +1,367 @@
+/*
+ * 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.adaptive
+
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.concurrent.TrieMap
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
+
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.exchange._
+import 
org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A root node to execute the query plan adaptively. It splits the query plan 
into independent
+ * stages and executes them in order according to their dependencies. The 
query stage
+ * materializes its output at the end. When one stage completes, the data 
statistics of the
+ * materialized output will be used to optimize the remainder of the query.
+ *
+ * To create query stages, we traverse the query tree bottom up. When we hit 
an exchange node,
+ * and if all the child query stages of this exchange node are materialized, 
we create a new
+ * query stage for this exchange node. The new stage is then materialized 
asynchronously once it
+ * is created.
+ *
+ * When one query stage finishes materialization, the rest query is 
re-optimized and planned based
+ * on the latest statistics provided by all materialized stages. Then we 
traverse the query plan
+ * again and create more stages if possible. After all stages have been 
materialized, we execute
+ * the rest of the plan.
+ */
+case class AdaptiveSparkPlanExec(
+initialPlan: SparkPlan,
+session: SparkSession,
+subqueryMap: Map[Long, ExecSubqueryExpression],
+stageCache: TrieMap[StructType, mutable.Buffer[(Exchange, 
QueryStageExec)]])
+  extends LeafExecNode {
+
+  def executedPlan: SparkPlan = currentPhysicalPlan
+
+  override def output: Seq[Attribute] = initialPlan.output
+
+  override def doCanonicalize(): SparkPlan = initialPlan.canonicalized
+
+  override def doExecute(): RDD[InternalRow] = {
+val events = new LinkedBlockingQueue[StageMaterializationEvent]()
+var result = createQueryStages(currentPhysicalPlan)
+while(!result.allChildStagesMaterialized) {
+  currentPhysicalPlan = result.newPlan
+  updateLogicalPlan(result.newStages)
+  onUpdatePlan()
+  result.newStages.map(_._2).foreach { stage =>
+stage.materialize().onComplete { res =>
+  if (res.isSuccess) {
+stage.resultOption = Some(res.get)
+events.offer(StageSuccess(stage))
+  } else {
+events.offer(StageFailure(stage, res.failed.get))
+  }
+}
+  }
+  // Wait on the next completed stage, which indicates new stats are 
available and probably
+  // new stages can be created. There might be other stages that finish at 
around the same
+  // time, so we process those stages too in order to reduce re-planning.
+  val nextMsg = events.take()
+  val rem = mutable.ArrayBuffer.empty[StageMaterializationEvent]
+  (Seq(nextMsg) ++ rem.toSeq).foreach{ e => e match
+{
+  case StageSuccess(stage) =>
+completedStages += stage.id
+  case StageFailure(stage, ex) =>
+throw new SparkException(
+  s"""
+ |Fail to materialize query stage ${stage.id}:
+ |${stage.plan.treeString}
+   

[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287548684
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
 ##
 @@ -0,0 +1,367 @@
+/*
+ * 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.adaptive
+
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.concurrent.TrieMap
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
+
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.exchange._
+import 
org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A root node to execute the query plan adaptively. It splits the query plan 
into independent
+ * stages and executes them in order according to their dependencies. The 
query stage
+ * materializes its output at the end. When one stage completes, the data 
statistics of the
+ * materialized output will be used to optimize the remainder of the query.
+ *
+ * To create query stages, we traverse the query tree bottom up. When we hit 
an exchange node,
+ * and if all the child query stages of this exchange node are materialized, 
we create a new
+ * query stage for this exchange node. The new stage is then materialized 
asynchronously once it
+ * is created.
+ *
+ * When one query stage finishes materialization, the rest query is 
re-optimized and planned based
+ * on the latest statistics provided by all materialized stages. Then we 
traverse the query plan
+ * again and create more stages if possible. After all stages have been 
materialized, we execute
+ * the rest of the plan.
+ */
+case class AdaptiveSparkPlanExec(
+initialPlan: SparkPlan,
+session: SparkSession,
+subqueryMap: Map[Long, ExecSubqueryExpression],
+stageCache: TrieMap[StructType, mutable.Buffer[(Exchange, 
QueryStageExec)]])
+  extends LeafExecNode {
+
+  def executedPlan: SparkPlan = currentPhysicalPlan
+
+  override def output: Seq[Attribute] = initialPlan.output
+
+  override def doCanonicalize(): SparkPlan = initialPlan.canonicalized
 
 Review comment:
   Why not currentPhysicalPlan.canonicalized but initialPlan.canonicalized?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287548644
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
 ##
 @@ -0,0 +1,367 @@
+/*
+ * 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.adaptive
+
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.concurrent.TrieMap
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
+
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.exchange._
+import 
org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A root node to execute the query plan adaptively. It splits the query plan 
into independent
+ * stages and executes them in order according to their dependencies. The 
query stage
 
 Review comment:
   It sounds a little weird that the stages are independent and executing them 
in order by their dependencies.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287549921
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/internal/SQLConf.scala
 ##
 @@ -292,6 +292,12 @@ object SQLConf {
   .bytesConf(ByteUnit.BYTE)
   .createWithDefault(64 * 1024 * 1024)
 
+  val RUNTIME_REOPTIMIZATION_ENABLED =
+buildConf("spark.sql.runtime.reoptimization.enabled")
+  .doc("When true, enable runtime query re-optimization.")
+  .booleanConf
+  .createWithDefault(false)
+
   val ADAPTIVE_EXECUTION_ENABLED = buildConf("spark.sql.adaptive.enabled")
 .doc("When true, enable adaptive query execution.")
 .booleanConf
 
 Review comment:
   We probably need to update the doc for this config. It isn't enabled when 
runtime query re-optimization is true.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287551380
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/SparkPlan.scala
 ##
 @@ -79,6 +81,34 @@ abstract class SparkPlan extends QueryPlan[SparkPlan] with 
Logging with Serializ
 super.makeCopy(newArgs)
   }
 
+  /**
+   * @return The logical plan this plan is linked to.
+   */
+  def logicalLink: Option[LogicalPlan] =
+getTagValue(SparkPlan.LOGICAL_PLAN_TAG)
+  .orElse(getTagValue(SparkPlan.LOGICAL_PLAN_INHERITED_TAG))
+
+  /**
+   * Set logical plan link recursively if unset.
+   */
+  def setLogicalLink(logicalPlan: LogicalPlan): Unit = {
+setLogicalLink(logicalPlan, false)
+  }
+
+  private def setLogicalLink(logicalPlan: LogicalPlan, inherited: Boolean = 
false): Unit = {
+if (logicalLink.isDefined) {
+  return
+}
+
+val tag = if (inherited) {
+  SparkPlan.LOGICAL_PLAN_INHERITED_TAG
+} else {
+  SparkPlan.LOGICAL_PLAN_TAG
+}
+setTagValue(tag, logicalPlan)
+children.foreach(_.setLogicalLink(logicalPlan, true))
 
 Review comment:
   Once we set inherited logical plan into children, we can't set logical plan 
into them? So only the top SparkPlan has its logical plan, all its children 
just have inherited logical plan?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287549166
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
 ##
 @@ -0,0 +1,367 @@
+/*
+ * 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.adaptive
+
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.concurrent.TrieMap
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
+
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.exchange._
+import 
org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A root node to execute the query plan adaptively. It splits the query plan 
into independent
+ * stages and executes them in order according to their dependencies. The 
query stage
+ * materializes its output at the end. When one stage completes, the data 
statistics of the
+ * materialized output will be used to optimize the remainder of the query.
+ *
+ * To create query stages, we traverse the query tree bottom up. When we hit 
an exchange node,
+ * and if all the child query stages of this exchange node are materialized, 
we create a new
+ * query stage for this exchange node. The new stage is then materialized 
asynchronously once it
+ * is created.
+ *
+ * When one query stage finishes materialization, the rest query is 
re-optimized and planned based
+ * on the latest statistics provided by all materialized stages. Then we 
traverse the query plan
+ * again and create more stages if possible. After all stages have been 
materialized, we execute
+ * the rest of the plan.
+ */
+case class AdaptiveSparkPlanExec(
+initialPlan: SparkPlan,
+session: SparkSession,
+subqueryMap: Map[Long, ExecSubqueryExpression],
+stageCache: TrieMap[StructType, mutable.Buffer[(Exchange, 
QueryStageExec)]])
+  extends LeafExecNode {
+
+  def executedPlan: SparkPlan = currentPhysicalPlan
+
+  override def output: Seq[Attribute] = initialPlan.output
+
+  override def doCanonicalize(): SparkPlan = initialPlan.canonicalized
+
+  override def doExecute(): RDD[InternalRow] = {
+val events = new LinkedBlockingQueue[StageMaterializationEvent]()
+var result = createQueryStages(currentPhysicalPlan)
+while(!result.allChildStagesMaterialized) {
+  currentPhysicalPlan = result.newPlan
+  updateLogicalPlan(result.newStages)
+  onUpdatePlan()
+  result.newStages.map(_._2).foreach { stage =>
+stage.materialize().onComplete { res =>
+  if (res.isSuccess) {
+stage.resultOption = Some(res.get)
+events.offer(StageSuccess(stage))
+  } else {
+events.offer(StageFailure(stage, res.failed.get))
+  }
+}
+  }
+  // Wait on the next completed stage, which indicates new stats are 
available and probably
+  // new stages can be created. There might be other stages that finish at 
around the same
+  // time, so we process those stages too in order to reduce re-planning.
+  val nextMsg = events.take()
+  val rem = mutable.ArrayBuffer.empty[StageMaterializationEvent]
+  (Seq(nextMsg) ++ rem.toSeq).foreach{ e => e match
 
 Review comment:
   Is any place updating to `rem`? Seems it remains empty?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries 

[GitHub] [spark] viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#discussion_r287551113
 
 

 ##
 File path: 
sql/core/src/main/scala/org/apache/spark/sql/execution/adaptive/AdaptiveSparkPlanExec.scala
 ##
 @@ -0,0 +1,367 @@
+/*
+ * 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.adaptive
+
+import java.util.concurrent.LinkedBlockingQueue
+
+import scala.collection.concurrent.TrieMap
+import scala.collection.mutable
+import scala.concurrent.{ExecutionContext, ExecutionContextExecutorService}
+
+import org.apache.spark.SparkException
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.Attribute
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, ReturnAnswer}
+import org.apache.spark.sql.catalyst.rules.{Rule, RuleExecutor}
+import org.apache.spark.sql.execution._
+import org.apache.spark.sql.execution.exchange._
+import 
org.apache.spark.sql.execution.ui.SparkListenerSQLAdaptiveExecutionUpdate
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.types.StructType
+import org.apache.spark.util.ThreadUtils
+
+/**
+ * A root node to execute the query plan adaptively. It splits the query plan 
into independent
+ * stages and executes them in order according to their dependencies. The 
query stage
+ * materializes its output at the end. When one stage completes, the data 
statistics of the
+ * materialized output will be used to optimize the remainder of the query.
+ *
+ * To create query stages, we traverse the query tree bottom up. When we hit 
an exchange node,
+ * and if all the child query stages of this exchange node are materialized, 
we create a new
+ * query stage for this exchange node. The new stage is then materialized 
asynchronously once it
+ * is created.
+ *
+ * When one query stage finishes materialization, the rest query is 
re-optimized and planned based
+ * on the latest statistics provided by all materialized stages. Then we 
traverse the query plan
+ * again and create more stages if possible. After all stages have been 
materialized, we execute
+ * the rest of the plan.
+ */
+case class AdaptiveSparkPlanExec(
+initialPlan: SparkPlan,
+session: SparkSession,
+subqueryMap: Map[Long, ExecSubqueryExpression],
+stageCache: TrieMap[StructType, mutable.Buffer[(Exchange, 
QueryStageExec)]])
+  extends LeafExecNode {
+
+  def executedPlan: SparkPlan = currentPhysicalPlan
+
+  override def output: Seq[Attribute] = initialPlan.output
+
+  override def doCanonicalize(): SparkPlan = initialPlan.canonicalized
+
+  override def doExecute(): RDD[InternalRow] = {
+val events = new LinkedBlockingQueue[StageMaterializationEvent]()
+var result = createQueryStages(currentPhysicalPlan)
+while(!result.allChildStagesMaterialized) {
+  currentPhysicalPlan = result.newPlan
+  updateLogicalPlan(result.newStages)
+  onUpdatePlan()
+  result.newStages.map(_._2).foreach { stage =>
+stage.materialize().onComplete { res =>
+  if (res.isSuccess) {
+stage.resultOption = Some(res.get)
+events.offer(StageSuccess(stage))
+  } else {
+events.offer(StageFailure(stage, res.failed.get))
+  }
+}
+  }
+  // Wait on the next completed stage, which indicates new stats are 
available and probably
+  // new stages can be created. There might be other stages that finish at 
around the same
+  // time, so we process those stages too in order to reduce re-planning.
+  val nextMsg = events.take()
+  val rem = mutable.ArrayBuffer.empty[StageMaterializationEvent]
+  (Seq(nextMsg) ++ rem.toSeq).foreach{ e => e match
+{
+  case StageSuccess(stage) =>
+completedStages += stage.id
+  case StageFailure(stage, ex) =>
+throw new SparkException(
+  s"""
+ |Fail to materialize query stage ${stage.id}:
+ |${stage.plan.treeString}
+   

[GitHub] [spark] AmplabJenkins removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24628: 
[SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495853162
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11033/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
SparkQA commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495853454
 
 
   **[Test build #105778 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105778/testReport)**
 for PR 24628 at commit 
[`b38c906`](https://github.com/apache/spark/commit/b38c906b5b0c2ebd8953342b079a7610b0d635ad).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24628: 
[SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495853160
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495853162
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11033/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495853160
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] wangyum commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
wangyum commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495852315
 
 
   retest this please


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24628: 
[SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495850292
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105777/
   Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24628: 
[SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495850290
 
 
   Merged build finished. Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495850290
 
 
   Merged build finished. Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495850292
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105777/
   Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
SparkQA removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495828488
 
 
   **[Test build #105777 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105777/testReport)**
 for PR 24628 at commit 
[`b38c906`](https://github.com/apache/spark/commit/b38c906b5b0c2ebd8953342b079a7610b0d635ad).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
SparkQA commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495850067
 
 
   **[Test build #105777 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105777/testReport)**
 for PR 24628 at commit 
[`b38c906`](https://github.com/apache/spark/commit/b38c906b5b0c2ebd8953342b079a7610b0d635ad).
* This patch **fails SparkR unit tests**.
* This patch merges cleanly.
* This patch adds no public classes.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] felixcheung commented on a change in pull request #24702: [Minor] [Kubernetes] Added retries on the connection to the driver for k8s

2019-05-24 Thread GitBox
felixcheung commented on a change in pull request #24702: [Minor] [Kubernetes] 
Added retries on the connection to the driver for k8s
URL: https://github.com/apache/spark/pull/24702#discussion_r287549781
 
 

 ##
 File path: 
core/src/main/scala/org/apache/spark/executor/CoarseGrainedExecutorBackend.scala
 ##
 @@ -283,7 +283,12 @@ private[spark] object CoarseGrainedExecutorBackend 
extends Logging {
 executorConf,
 new SecurityManager(executorConf),
 clientMode = true)
-  val driver = fetcher.setupEndpointRefByURI(arguments.driverUrl)
+
+val driver = retry(3) {
+  fetcher.setupEndpointRefByURI(arguments.driverUrl)
+}
 
 Review comment:
   yes, that's what I mean. in case of shared class, the PR should tag all of 
them


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] felixcheung commented on a change in pull request #24414: [SPARK-22044][SQL] Add `cost` and `codegen` arguments to `explain`

2019-05-24 Thread GitBox
felixcheung commented on a change in pull request #24414: [SPARK-22044][SQL] 
Add `cost` and `codegen` arguments to `explain`
URL: https://github.com/apache/spark/pull/24414#discussion_r287549710
 
 

 ##
 File path: R/pkg/R/DataFrame.R
 ##
 @@ -136,7 +136,15 @@ setMethod("schema",
 
 #' Explain
 #'
-#' Print the logical and physical Catalyst plans to the console for debugging.
+#' Print the Catalyst plans to the console for debugging.
+#'
+#' With no (or, all FALSE) options, this prints the physical
+#' plan. The options are mutually exclusive: at most one can be
+#' true.
+#'
+#' @param extended print the logical plans as well as the physical plans
+#' @param codegen print the generated code for whole-stage codegen
+#' @param cost print the optimized logical plan with operator costs
 
 Review comment:
   It means all the param should be added there 
(https://github.com/apache/spark/blob/beba93c562cc98f6ebe799830f55e54d7c69fa0d/R/pkg/R/generics.R#L467)


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jzhuge commented on a change in pull request #24686: [SPARK-27813][SQL] DataSourceV2: Add DropTable logical operation

2019-05-24 Thread GitBox
jzhuge commented on a change in pull request #24686: [SPARK-27813][SQL] 
DataSourceV2: Add DropTable logical operation
URL: https://github.com/apache/spark/pull/24686#discussion_r287549180
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/analysis/Analyzer.scala
 ##
 @@ -96,19 +95,13 @@ object AnalysisContext {
 class Analyzer(
 catalog: SessionCatalog,
 conf: SQLConf,
-maxIterations: Int,
-override val lookupCatalog: Option[(String) => CatalogPlugin] = None)
-  extends RuleExecutor[LogicalPlan] with CheckAnalysis with LookupCatalog {
+maxIterations: Int)
+  extends RuleExecutor[LogicalPlan] with CheckAnalysis {
 
 Review comment:
   Will keep it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495832542
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105776/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495832537
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to 
do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495832542
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105776/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to 
do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495832537
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA removed a comment on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
SparkQA removed a comment on issue #24706: [SPARK-23128][SQL] A new approach to 
do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495815908
 
 
   **[Test build #105776 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105776/testReport)**
 for PR 24706 at commit 
[`07ea123`](https://github.com/apache/spark/commit/07ea123f27e931c4587f6cb26c53d8c3176926d3).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer edited a comment on issue #24372: [SPARK-27462][SQL] Enhance insert into hive table that could choose some columns in target table flexibly.

2019-05-24 Thread GitBox
beliefer edited a comment on issue #24372: [SPARK-27462][SQL] Enhance insert 
into hive table that could choose some columns in target table flexibly.
URL: https://github.com/apache/spark/pull/24372#issuecomment-495832276
 
 
   @mgaido91 I approve your opinion this feature not free and need more broad 
discussion. Spark sql already exists this issue,  some syntax (e.g create hive 
table, create data source table, insert overwrite and so on.) is the same as 
the syntax this PR to implement. In a production environment, user switch 
source will increase learning and maintenance costs. 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] beliefer commented on issue #24372: [SPARK-27462][SQL] Enhance insert into hive table that could choose some columns in target table flexibly.

2019-05-24 Thread GitBox
beliefer commented on issue #24372: [SPARK-27462][SQL] Enhance insert into hive 
table that could choose some columns in target table flexibly.
URL: https://github.com/apache/spark/pull/24372#issuecomment-495832276
 
 
   @mgaido91 I approve your opinion this feature not free and need more broad 
discussion. Spark sql already exists this issue,  some syntax (e.g create hive 
table, create data source table, insert overwrite and so on.) is the as the 
syntax this PR to implement. In a production environment, user switch source 
will increase learning and maintenance costs. 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
SparkQA commented on issue #24706: [SPARK-23128][SQL] A new approach to do 
adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495832220
 
 
   **[Test build #105776 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105776/testReport)**
 for PR 24706 at commit 
[`07ea123`](https://github.com/apache/spark/commit/07ea123f27e931c4587f6cb26c53d8c3176926d3).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds the following public classes _(experimental)_:
 * `case class AdaptiveSparkPlanExec(`
 * `case class StageSuccess(stage: QueryStageExec) extends 
StageMaterializationEvent`
 * `case class StageFailure(stage: QueryStageExec, error: Throwable) 
extends StageMaterializationEvent`
 * `case class InsertAdaptiveSparkPlan(session: SparkSession) extends 
Rule[SparkPlan] `
 * `case class LogicalQueryStage(`
 * `case class PlanAdaptiveSubqueries(`
 * `abstract class QueryStageExec extends LeafExecNode `
 * `case class ShuffleQueryStageExec(`
 * `case class BroadcastQueryStageExec(`
 * `case class ReusedQueryStageExec(`
 * `case class SparkListenerSQLAdaptiveExecutionUpdate(`


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
SparkQA commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495828488
 
 
   **[Test build #105777 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105777/testReport)**
 for PR 24628 at commit 
[`b38c906`](https://github.com/apache/spark/commit/b38c906b5b0c2ebd8953342b079a7610b0d635ad).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495828300
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11032/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] 
hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495828299
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24628: 
[SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495828299
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24628: [SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24628: 
[SPARK-27749][SQL][test-hadoop3.2] hadoop-3.2 support hive-thriftserver
URL: https://github.com/apache/spark/pull/24628#issuecomment-495828300
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11032/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] viirya commented on a change in pull request #24699: [SPARK-27666][CORE] Stop PythonRunner's WriteThread immediately when task finishes

2019-05-24 Thread GitBox
viirya commented on a change in pull request #24699: [SPARK-27666][CORE] Stop 
PythonRunner's WriteThread immediately when task finishes
URL: https://github.com/apache/spark/pull/24699#discussion_r287546925
 
 

 ##
 File path: core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
 ##
 @@ -180,6 +180,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
 def shutdownOnTaskCompletion() {
   assert(context.isCompleted)
   this.interrupt()
+  this.join()
 
 Review comment:
   It causes a deadlock on the lock on task context.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dilipbiswal commented on a change in pull request #24692: [SPARK-27824][SQL] Make rule EliminateResolvedHint idempotent

2019-05-24 Thread GitBox
dilipbiswal commented on a change in pull request #24692: [SPARK-27824][SQL] 
Make rule EliminateResolvedHint idempotent
URL: https://github.com/apache/spark/pull/24692#discussion_r287545659
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala
 ##
 @@ -29,7 +29,7 @@ object EliminateResolvedHint extends Rule[LogicalPlan] {
   // is using transformUp rather than resolveOperators.
   def apply(plan: LogicalPlan): LogicalPlan = {
 val pulledUp = plan transformUp {
-  case j: Join =>
+  case j: Join if j.hint == JoinHint.NONE =>
 
 Review comment:
   @maryannxue Thanks for explanation.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dongjoon-hyun closed pull request #24694: [SPARK-27830][CORE][UI] Show Spark version at app lists of Spark History UI

2019-05-24 Thread GitBox
dongjoon-hyun closed pull request #24694: [SPARK-27830][CORE][UI] Show Spark 
version at app lists of Spark History UI
URL: https://github.com/apache/spark/pull/24694
 
 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve 
logic for timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495818494
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105774/
   Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve 
logic for timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495818492
 
 
   Merged build finished. Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for 
timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495818494
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105774/
   Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA removed a comment on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
SparkQA removed a comment on issue #24704: [SPARK-20286][core] Improve logic 
for timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495807196
 
 
   **[Test build #105774 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105774/testReport)**
 for PR 24704 at commit 
[`05b6802`](https://github.com/apache/spark/commit/05b68025d40591c87b2e8584555e9658226ddef0).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for 
timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495818492
 
 
   Merged build finished. Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
SparkQA commented on issue #24704: [SPARK-20286][core] Improve logic for timing 
out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495818398
 
 
   **[Test build #105774 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105774/testReport)**
 for PR 24704 at commit 
[`05b6802`](https://github.com/apache/spark/commit/05b68025d40591c87b2e8584555e9658226ddef0).
* This patch **fails Spark unit tests**.
* This patch merges cleanly.
* This patch adds no public classes.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] HyukjinKwon edited a comment on issue #24675: [SPARK-27803][SQL][PYTHON] Fix column pruning for Python UDF

2019-05-24 Thread GitBox
HyukjinKwon edited a comment on issue #24675: [SPARK-27803][SQL][PYTHON] Fix 
column pruning for Python UDF
URL: https://github.com/apache/spark/pull/24675#issuecomment-495591310
 
 
   BTW, just to be sync'ed with you too @BryanCutler, @viirya and @icexelloss, 
I am planning to add a bunch of tests specific to regular Python UDF and Pandas 
Scalar UDF, which are possibly able to reused to Scala UDF too - I am trying to 
find a way to deduplicate as much as possible. I hopefully it makes sense to 
you guys.
   
   This special rule `ExtractPythonUDF[s|FromAggregate]` has unevaluable 
expressions that always has to be wrapped with special plans. Seems like we 
remove some hacks now but I think we're not sure about the coverage.
   
   I think we started to observe those issues since we turned those Python ones 
from physical plans to logical plans, which was (I think) right fix but 
couldn't catch many cases like this. My idea is basically to share (or 
partially duplicate) *.sql files for Python / Pandas / Scala UDFs - hope this 
idea prevents such issues in the future.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495815714
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
SparkQA commented on issue #24706: [SPARK-23128][SQL] A new approach to do 
adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495815908
 
 
   **[Test build #105776 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105776/testReport)**
 for PR 24706 at commit 
[`07ea123`](https://github.com/apache/spark/commit/07ea123f27e931c4587f6cb26c53d8c3176926d3).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24706: [SPARK-23128][SQL] A new 
approach to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495815716
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11031/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jkbradley commented on a change in pull request #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
jkbradley commented on a change in pull request #24705: [SPARK-22340][PYTHON] 
Save localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#discussion_r287539792
 
 

 ##
 File path: python/pyspark/tests/test_rdd.py
 ##
 @@ -761,6 +761,66 @@ def fail_last(x):
 for i in range(4):
 self.assertEqual(i, next(it))
 
+def test_multiple_group_jobs(self):
+# SPARK-22340 test setJobGroup for multiple job groups
+
+import threading
+import time
+
+group_A_name = "group_A"
+group_B_name = "group_B"
+
+def map_func(x):
+time.sleep(x)
+return x
+
+num_threads = 4
+thread_list = []
+# an array which record whether job is cancelled.
+# the index of the array is the thread index which job run in.
+is_job_cancelled = [False for x in range(num_threads)]
+
+def run_job(job_group, index):
+try:
+self.sc.setJobGroup(job_group, "test rdd collect with setting 
job group")
+result = self.sc.parallelize([3]).map(map_func).collect()
+is_job_cancelled[index] = False
+return result
+except Exception as e:
+is_job_cancelled[index] = True
+print("{} with message: {}".format(type(e).__name__, str(e)))
+return None
+
+def launch_job_thread(job_group, index):
+thread = threading.Thread(target=run_job, args=(job_group, index))
+thread.start()
+return thread
+
+# test job succeeded when not cancelled.
+run_job(group_A_name, 0)
+self.assertFalse(is_job_cancelled[0], "job didn't succeeded.")
+
+for i in range(num_threads):
 
 Review comment:
   I'd explain what this is testing in a comment.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jkbradley commented on a change in pull request #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
jkbradley commented on a change in pull request #24705: [SPARK-22340][PYTHON] 
Save localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#discussion_r287539153
 
 

 ##
 File path: core/src/main/scala/org/apache/spark/api/python/PythonRDD.scala
 ##
 @@ -159,7 +159,12 @@ private[spark] object PythonRDD extends Logging {
* @return 2-tuple (as a Java array) with the port number of a local socket 
which serves the
* data collected from this job, and the secret for authentication.
*/
-  def collectAndServe[T](rdd: RDD[T]): Array[Any] = {
+  def collectAndServe[T](
 
 Review comment:
   I'll let @ueshin judge, but my guess is that, even though this is a 
"private" API, we'll want to add a new collectAndServe with the 2 arguments, 
leaving the old one in case 3rd-party libraries use the private API.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] HyukjinKwon edited a comment on issue #24675: [SPARK-27803][SQL][PYTHON] Fix column pruning for Python UDF

2019-05-24 Thread GitBox
HyukjinKwon edited a comment on issue #24675: [SPARK-27803][SQL][PYTHON] Fix 
column pruning for Python UDF
URL: https://github.com/apache/spark/pull/24675#issuecomment-495591310
 
 
   BTW, just to be sync'ed with you too @BryanCutler, @viirya and @icexelloss, 
I am planning to add a bunch of tests specific to regular Python UDF and Pandas 
Scalar UDF, which are possibly able to reused to Scala UDF too - I am trying to 
find a way to duplicate as much as possible. I hopefully it makes sense to you 
guys.
   
   This special rule `ExtractPythonUDF[s|FromAggregate]` has unevaluable 
expressions that always has to be wrapped with special plans. Seems like we 
remove some hacks now but I think we're not sure about the coverage.
   
   I think we started to observe those issues since we turned those Python ones 
from physical plans to logical plans, which was (I think) right fix but 
couldn't catch many cases like this. My idea is basically to share (or 
partially duplicate) *.sql files for Python / Pandas / Scala UDFs - hope this 
idea prevents such issues in the future.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to 
do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495815716
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11031/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24706: [SPARK-23128][SQL] A new approach to 
do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706#issuecomment-495815714
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maryannxue opened a new pull request #24706: [SPARK-23128][SQL] A new approach to do adaptive execution in Spark SQL

2019-05-24 Thread GitBox
maryannxue opened a new pull request #24706: [SPARK-23128][SQL] A new approach 
to do adaptive execution in Spark SQL
URL: https://github.com/apache/spark/pull/24706
 
 
   ## What changes were proposed in this pull request?
   
   Implemented a new SparkPlan that executes the query adaptively. It splits 
the query plan into independent stages and executes them in order according to 
their dependencies. The query stage materializes its output at the end. When 
one stage completes, the data statistics of the materialized output will be 
used to optimize the remainder of the query.
   
   ## How was this patch tested?
   
   Added new UT.
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495815012
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11030/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495815012
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11030/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495815010
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495815010
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814647
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105775/
   Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
SparkQA removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814452
 
 
   **[Test build #105775 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105775/testReport)**
 for PR 24705 at commit 
[`937a4a6`](https://github.com/apache/spark/commit/937a4a6b761f0de3a3d277066099c16fda4b9e8d).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814643
 
 
   Merged build finished. Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814643
 
 
   Merged build finished. Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814647
 
 
   Test FAILed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105775/
   Test FAILed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
SparkQA commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties 
in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814641
 
 
   **[Test build #105775 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105775/testReport)**
 for PR 24705 at commit 
[`937a4a6`](https://github.com/apache/spark/commit/937a4a6b761f0de3a3d277066099c16fda4b9e8d).
* This patch **fails Python style tests**.
* This patch merges cleanly.
* This patch adds no public classes.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
SparkQA commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties 
in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814452
 
 
   **[Test build #105775 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105775/testReport)**
 for PR 24705 at commit 
[`937a4a6`](https://github.com/apache/spark/commit/937a4a6b761f0de3a3d277066099c16fda4b9e8d).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jkbradley commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
jkbradley commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties 
in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814216
 
 
   CC @ueshin 


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495810674
 
 
   Can one of the admins verify this patch?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jkbradley commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
jkbradley commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties 
in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495814177
 
 
   add to whitelist


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495810346
 
 
   Can one of the admins verify this patch?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495810674
 
 
   Can one of the admins verify this patch?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495810265
 
 
   Can one of the admins verify this patch?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495810346
 
 
   Can one of the admins verify this patch?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705#issuecomment-495810265
 
 
   Can one of the admins verify this patch?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] lu-wang-dl opened a new pull request #24705: [SPARK-22340][PYTHON] Save localProperties in thread.local

2019-05-24 Thread GitBox
lu-wang-dl opened a new pull request #24705: [SPARK-22340][PYTHON] Save 
localProperties in thread.local
URL: https://github.com/apache/spark/pull/24705
 
 
   ## What changes were proposed in this pull request?
   
   - Add `threading.local()` in `SparkContext`
   - Save the value to thread local variables when calling `setLocalProperty`, 
`setJobGroup`,  and `setJobDescription`
   - Add new API `getLocalProperties`
   - Send local properties to jvm side when calling `collect`
   - On jvm side `setLocalProperty` in `collectAndServe`
   
   ## How was this patch tested?
   
   Add one unit test in test_rdd.py
   
   Please review https://spark.apache.org/contributing.html before opening a 
pull request.
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] vanzin commented on a change in pull request #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
vanzin commented on a change in pull request #24704: [SPARK-20286][core] 
Improve logic for timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#discussion_r287534475
 
 

 ##
 File path: 
core/src/test/scala/org/apache/spark/deploy/ExternalShuffleServiceDbSuite.scala
 ##
 @@ -70,10 +71,12 @@ class ExternalShuffleServiceDbSuite extends SparkFunSuite {
 }
   }
 
+  def shuffleServiceConf: SparkConf = 
sparkConf.clone().set(SHUFFLE_SERVICE_PORT, 0)
 
 Review comment:
   This change isn't exactly related; but this test was failing when I forgot a 
shuffle service running locally when running tests for the rest of this PR. So 
might as well fix this.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24553: [SPARK-27604][SQL] Enhance constant propagation

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24553: [SPARK-27604][SQL] Enhance 
constant propagation
URL: https://github.com/apache/spark/pull/24553#issuecomment-495807928
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24553: [SPARK-27604][SQL] Enhance constant propagation

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24553: [SPARK-27604][SQL] Enhance 
constant propagation
URL: https://github.com/apache/spark/pull/24553#issuecomment-495807933
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105769/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24553: [SPARK-27604][SQL] Enhance constant propagation

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24553: [SPARK-27604][SQL] Enhance constant 
propagation
URL: https://github.com/apache/spark/pull/24553#issuecomment-495807928
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24553: [SPARK-27604][SQL] Enhance constant propagation

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24553: [SPARK-27604][SQL] Enhance constant 
propagation
URL: https://github.com/apache/spark/pull/24553#issuecomment-495807933
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105769/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA removed a comment on issue #24553: [SPARK-27604][SQL] Enhance constant propagation

2019-05-24 Thread GitBox
SparkQA removed a comment on issue #24553: [SPARK-27604][SQL] Enhance constant 
propagation
URL: https://github.com/apache/spark/pull/24553#issuecomment-495768977
 
 
   **[Test build #105769 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105769/testReport)**
 for PR 24553 at commit 
[`6bccd84`](https://github.com/apache/spark/commit/6bccd84e9cdc9546d9311e06d6837981762065a3).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24553: [SPARK-27604][SQL] Enhance constant propagation

2019-05-24 Thread GitBox
SparkQA commented on issue #24553: [SPARK-27604][SQL] Enhance constant 
propagation
URL: https://github.com/apache/spark/pull/24553#issuecomment-495807651
 
 
   **[Test build #105769 has 
finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105769/testReport)**
 for PR 24553 at commit 
[`6bccd84`](https://github.com/apache/spark/commit/6bccd84e9cdc9546d9311e06d6837981762065a3).
* This patch passes all tests.
* This patch merges cleanly.
* This patch adds no public classes.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve 
logic for timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495806952
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11029/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24704: [SPARK-20286][core] Improve 
logic for timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495806951
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] SparkQA commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
SparkQA commented on issue #24704: [SPARK-20286][core] Improve logic for timing 
out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495807196
 
 
   **[Test build #105774 has 
started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/105774/testReport)**
 for PR 24704 at commit 
[`05b6802`](https://github.com/apache/spark/commit/05b68025d40591c87b2e8584555e9658226ddef0).


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for 
timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495806951
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24704: [SPARK-20286][core] Improve logic for 
timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495806952
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   
https://amplab.cs.berkeley.edu/jenkins//job/testing-k8s-prb-make-spark-distribution-unified/11029/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] vanzin commented on issue #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
vanzin commented on issue #24704: [SPARK-20286][core] Improve logic for timing 
out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704#issuecomment-495806323
 
 
   @dhruve I cleaned up the code I had lying around and made it work on top of 
the existing EAM. This PR is a replacement for #22015.
   
   @attilapiros I removed some code you added recently as part of this patch.
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] vanzin opened a new pull request #24704: [SPARK-20286][core] Improve logic for timing out executors in dynamic allocation.

2019-05-24 Thread GitBox
vanzin opened a new pull request #24704: [SPARK-20286][core] Improve logic for 
timing out executors in dynamic allocation.
URL: https://github.com/apache/spark/pull/24704
 
 
   This change refactors the portions of the ExecutorAllocationManager class 
that
   track executor state into a new class, to achieve a few goals:
   
   - make the code easier to understand
   - better separate concerns (task backlog vs. executor state)
   - less synchronization between event and allocation threads
   - less coupling between the allocation code and executor state tracking
   
   The executor tracking code was moved to a new class (ExecutorMonitor) that
   encapsulates all the logic of tracking what happens to executors and when
   they can be timed out. The logic to actually remove the executors remains
   in the EAM, since it still requires information that is not tracked by the
   new executor monitor code.
   
   In the executor monitor itself, of interest, specifically, is a change in
   how cached blocks are tracked; instead of polling the block manager, the
   monitor now uses events to track which executors have cached blocks, and
   is able to detect also unpersist events and adjust the time when the executor
   should be removed accordingly. (That's the bug mentioned in the PR title.)
   
   Because of the refactoring, a few tests in the old EAM test suite were 
removed,
   since they're now covered by the newly added test suite. The EAM suite was
   also changed a little bit to not instantiate a SparkContext every time. This
   allowed some cleanup, and the tests also run faster.
   
   Tested with new and updated unit tests, and with multiple TPC-DS workloads
   running with dynamic allocation on; also some manual tests for the caching
   behavior.
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] jiangxb1987 commented on a change in pull request #24699: [SPARK-27666][CORE] Stop PythonRunner's WriteThread immediately when task finishes

2019-05-24 Thread GitBox
jiangxb1987 commented on a change in pull request #24699: [SPARK-27666][CORE] 
Stop PythonRunner's WriteThread immediately when task finishes
URL: https://github.com/apache/spark/pull/24699#discussion_r287529871
 
 

 ##
 File path: core/src/main/scala/org/apache/spark/api/python/PythonRunner.scala
 ##
 @@ -180,6 +180,7 @@ private[spark] abstract class BasePythonRunner[IN, OUT](
 def shutdownOnTaskCompletion() {
   assert(context.isCompleted)
   this.interrupt()
+  this.join()
 
 Review comment:
   This just blocks the current Thread and wait for interrupt to take effect?


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] gatorsmile closed pull request #24681: [SPARK-27809] [SQL] Make optional clauses order insensitive for CREATE DATABASE/VIEW SQL statement

2019-05-24 Thread GitBox
gatorsmile closed pull request #24681: [SPARK-27809] [SQL] Make optional 
clauses order insensitive for CREATE DATABASE/VIEW SQL statement
URL: https://github.com/apache/spark/pull/24681
 
 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] gatorsmile commented on issue #24681: [SPARK-27809] [SQL] Make optional clauses order insensitive for CREATE DATABASE/VIEW SQL statement

2019-05-24 Thread GitBox
gatorsmile commented on issue #24681: [SPARK-27809] [SQL] Make optional clauses 
order insensitive for CREATE DATABASE/VIEW SQL statement
URL: https://github.com/apache/spark/pull/24681#issuecomment-495802889
 
 
   Thanks! Merged to master.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dbtsai commented on issue #24694: [SPARK-27830][CORE][UI] Show Spark version at app lists of Spark History UI

2019-05-24 Thread GitBox
dbtsai commented on issue #24694: [SPARK-27830][CORE][UI] Show Spark version at 
app lists of Spark History UI
URL: https://github.com/apache/spark/pull/24694#issuecomment-495802237
 
 
   LGTM.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] HyukjinKwon commented on issue #24700: [SPARK-27834][SQL][R][PYTHON] Make separate PySpark/SparkR vectorization configurations

2019-05-24 Thread GitBox
HyukjinKwon commented on issue #24700: [SPARK-27834][SQL][R][PYTHON] Make 
separate PySpark/SparkR vectorization configurations
URL: https://github.com/apache/spark/pull/24700#issuecomment-495800583
 
 
   Yes.. I am not sure what we should name tho. If we name it spark.pyspark 
then it's usually spark conf at SpsrkContext. I was thinking it makes sense to 
spark.sql.pyspark too in a way because it works closely with SQL.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dbtsai closed pull request #24682: [SPARK-27838][SQL] Support user provided non-nullable avro schema for nullable catalyst schema without any null record

2019-05-24 Thread GitBox
dbtsai closed pull request #24682: [SPARK-27838][SQL] Support user provided 
non-nullable avro schema for nullable catalyst schema without any null record
URL: https://github.com/apache/spark/pull/24682
 
 
   


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] dbtsai commented on issue #24682: [SPARK-27838][SQL] Support user provided non-nullable avro schema for nullable catalyst schema without any null record

2019-05-24 Thread GitBox
dbtsai commented on issue #24682: [SPARK-27838][SQL] Support user provided 
non-nullable avro schema for nullable catalyst schema without any null record
URL: https://github.com/apache/spark/pull/24682#issuecomment-495796877
 
 
   Thanks all again. Merged into master.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] maryannxue commented on a change in pull request #24653: [SPARK-27783][SQL] Add customizable hint error handler

2019-05-24 Thread GitBox
maryannxue commented on a change in pull request #24653: [SPARK-27783][SQL] Add 
customizable hint error handler
URL: https://github.com/apache/spark/pull/24653#discussion_r287522772
 
 

 ##
 File path: 
sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/EliminateResolvedHint.scala
 ##
 @@ -37,7 +41,7 @@ object EliminateResolvedHint extends Rule[LogicalPlan] {
 }
 pulledUp.transformUp {
   case h: ResolvedHint =>
-handleInvalidHintInfo(h.hints)
+hintErrorHandler.handleJoinNotFoundForJoinHint(h.hints)
 
 Review comment:
   The hope is we'll be able to recognize the specific hint error in this rule 
if anything else. Say if a XYZ-hint has to be associated with XYZ-node, we 
should another error assigned to it.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24682: [SPARK-27838][SQL] Support user provided non-nullable avro schema for nullable catalyst schema without any null record

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24682: [SPARK-27838][SQL] Support 
user provided non-nullable avro schema for nullable catalyst schema without any 
null record
URL: https://github.com/apache/spark/pull/24682#issuecomment-495794869
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105773/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins removed a comment on issue #24682: [SPARK-27838][SQL] Support user provided non-nullable avro schema for nullable catalyst schema without any null record

2019-05-24 Thread GitBox
AmplabJenkins removed a comment on issue #24682: [SPARK-27838][SQL] Support 
user provided non-nullable avro schema for nullable catalyst schema without any 
null record
URL: https://github.com/apache/spark/pull/24682#issuecomment-495794861
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24682: [SPARK-27838][SQL] Support user provided non-nullable avro schema for nullable catalyst schema without any null record

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24682: [SPARK-27838][SQL] Support user 
provided non-nullable avro schema for nullable catalyst schema without any null 
record
URL: https://github.com/apache/spark/pull/24682#issuecomment-495794869
 
 
   Test PASSed.
   Refer to this link for build results (access rights to CI server needed): 
   https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/105773/
   Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



[GitHub] [spark] AmplabJenkins commented on issue #24682: [SPARK-27838][SQL] Support user provided non-nullable avro schema for nullable catalyst schema without any null record

2019-05-24 Thread GitBox
AmplabJenkins commented on issue #24682: [SPARK-27838][SQL] Support user 
provided non-nullable avro schema for nullable catalyst schema without any null 
record
URL: https://github.com/apache/spark/pull/24682#issuecomment-495794861
 
 
   Merged build finished. Test PASSed.


This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


With regards,
Apache Git Services

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



  1   2   3   4   5   >