[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user wzhfy commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110089743 --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala --- @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf._ + + +class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBase { + + override val conf = new SQLConf().copy( +CASE_SENSITIVE -> true, +CBO_ENABLED -> true, +JOIN_REORDER_ENABLED -> true, +STARSCHEMA_DETECTION -> true, +JOIN_REORDER_DP_STAR_FILTER -> true) + + object Optimize extends RuleExecutor[LogicalPlan] { +val batches = + Batch("Operator Optimizations", FixedPoint(100), +CombineFilters, +PushDownPredicate, +ReorderJoin(conf), +PushPredicateThroughJoin, +ColumnPruning, +CollapseProject) :: +Batch("Join Reorder", Once, + CostBasedJoinReorder(conf)) :: Nil + } + + private val columnInfo: AttributeMap[ColumnStat] = AttributeMap(Seq( +// F1 (fact table) +attr("f1_fk1") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_fk2") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_fk3") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_c1") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_c2") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), + +// D1 (dimension) +attr("d1_pk") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(100), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d1_c2") -> ColumnStat(distinctCount = 50, min = Some(1), max = Some(20), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d1_c3") -> ColumnStat(distinctCount = 50, min = Some(1), max = Some(20), + nullCount = 0, avgLen = 4, maxLen = 4), + +// D2 (dimension) +attr("d2_pk") -> ColumnStat(distinctCount = 20, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d2_c2") -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d2_c3") -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), + +// D3 (dimension) +attr("d3_pk") -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d3_c2") -> ColumnStat(distinctCount = 5, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d3_c3") -> ColumnStat(distinctCount = 5, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), + +// T1 +attr("t1_c1") -> ColumnStat(distinctCount = 20, min = Some(1), max = Some(5), + nullCount = 1, avgLen = 4, maxLen = 4), +attr("t1_c2") -> ColumnStat(distinctCount = 1
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user wzhfy commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110089115 --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala --- @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf._ + + +class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBase { + + override val conf = new SQLConf().copy( +CASE_SENSITIVE -> true, +CBO_ENABLED -> true, +JOIN_REORDER_ENABLED -> true, +STARSCHEMA_DETECTION -> true, +JOIN_REORDER_DP_STAR_FILTER -> true) + + object Optimize extends RuleExecutor[LogicalPlan] { +val batches = + Batch("Operator Optimizations", FixedPoint(100), +CombineFilters, +PushDownPredicate, +ReorderJoin(conf), +PushPredicateThroughJoin, +ColumnPruning, +CollapseProject) :: +Batch("Join Reorder", Once, + CostBasedJoinReorder(conf)) :: Nil + } + + private val columnInfo: AttributeMap[ColumnStat] = AttributeMap(Seq( +// F1 (fact table) +attr("f1_fk1") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_fk2") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_fk3") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_c1") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("f1_c2") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), + nullCount = 0, avgLen = 4, maxLen = 4), + +// D1 (dimension) +attr("d1_pk") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(100), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d1_c2") -> ColumnStat(distinctCount = 50, min = Some(1), max = Some(20), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d1_c3") -> ColumnStat(distinctCount = 50, min = Some(1), max = Some(20), + nullCount = 0, avgLen = 4, maxLen = 4), + +// D2 (dimension) +attr("d2_pk") -> ColumnStat(distinctCount = 20, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d2_c2") -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d2_c3") -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), + +// D3 (dimension) +attr("d3_pk") -> ColumnStat(distinctCount = 10, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d3_c2") -> ColumnStat(distinctCount = 5, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), +attr("d3_c3") -> ColumnStat(distinctCount = 5, min = Some(1), max = Some(5), + nullCount = 0, avgLen = 4, maxLen = 4), + +// T1 --- End diff -- add some comment to indicate it's a table not in a star schema? --- If your project is set up for it, you can reply to this email and have your reply appear on Gi
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user wzhfy commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110089871 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala --- @@ -327,3 +345,104 @@ object JoinReorderDP extends PredicateHelper with Logging { case class Cost(card: BigInt, size: BigInt) { def +(other: Cost): Cost = Cost(this.card + other.card, this.size + other.size) } + +/** + * Implements optional filters to reduce the search space for join enumeration. + * + * 1) Star-join filters: Plan star-joins together since they are assumed + *to have an optimal execution based on their RI relationship. + * 2) Cartesian products: Defer their planning later in the graph to avoid + *large intermediate results (expanding joins, in general). + * 3) Composite inners: Don't generate "bushy tree" plans to avoid materializing + * intermediate results. + * + * Filters (2) and (3) are not implemented. + */ +case class JoinReorderDPFilters(conf: SQLConf) extends PredicateHelper { + /** + * Builds join graph information to be used by the filtering strategies. + * Currently, it builds the sets of star/non-star joins. + * It can be extended with the sets of connected/unconnected joins, which + * can be used to filter Cartesian products. + */ + def buildJoinGraphInfo( + items: Seq[LogicalPlan], + conditions: Set[Expression], + planIndex: Seq[(LogicalPlan, Int)]): Option[JoinGraphInfo] = { + +// Compute the tables in a star-schema relationship. +val starJoin = StarSchemaDetection(conf).findStarJoins(items, conditions.toSeq) +val nonStarJoin = items.filterNot(starJoin.contains(_)) + +if (starJoin.nonEmpty && nonStarJoin.nonEmpty) { + val (starInt, nonStarInt) = planIndex.collect { +case (p, i) if starJoin.contains(p) => + (Some(i), None) +case (p, i) if nonStarJoin.contains(p) => + (None, Some(i)) +case _ => + (None, None) + }.unzip + Some(JoinGraphInfo(starInt.flatten.toSet, nonStarInt.flatten.toSet)) +} else { + // Nothing interesting to return. + None +} + } + + /** + * Applies star-join filter. + * + * Given the outer/inner and the star/non-star sets, + * the following plan combinations are allowed: + * 1) (outer U inner) is a subset of star-join + * 2) star-join is a subset of (outer U inner) + * 3) (outer U inner) is a subset of non star-join + * + * It assumes the sets are disjoint. + * + * Example query graph: + * + * t1 d1 - t2 - t3 + * \ / + * f1 + * | + * d2 + * + * star: {d1, f1, d2} + * non-star: {t2, t1, t3} + * + * level 0: (f1 ), (d2 ), (t3 ), (d1 ), (t1 ), (t2 ) + * level 1: {t3 t2 }, {f1 d2 }, {f1 d1 } + * level 2: {d2 f1 d1 } + * level 3: {t1 d1 f1 d2 }, {t2 d1 f1 d2 } + * level 4: {d1 t2 f1 t1 d2 }, {d1 t3 t2 f1 d2 } + * level 5: {d1 t3 t2 f1 t1 d2 } + */ + def starJoinFilter( + outer: Set[Int], + inner: Set[Int], + filters: JoinGraphInfo) : Boolean = { +val starJoins = filters.starJoins +val nonStarJoins = filters.nonStarJoins +val join = outer.union(inner) + +// Disjoint sets +outer.intersect(inner).isEmpty && --- End diff -- +1 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user wzhfy commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110088813 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala --- @@ -327,3 +345,104 @@ object JoinReorderDP extends PredicateHelper with Logging { case class Cost(card: BigInt, size: BigInt) { def +(other: Cost): Cost = Cost(this.card + other.card, this.size + other.size) } + +/** + * Implements optional filters to reduce the search space for join enumeration. + * + * 1) Star-join filters: Plan star-joins together since they are assumed + *to have an optimal execution based on their RI relationship. + * 2) Cartesian products: Defer their planning later in the graph to avoid + *large intermediate results (expanding joins, in general). + * 3) Composite inners: Don't generate "bushy tree" plans to avoid materializing + * intermediate results. + * + * Filters (2) and (3) are not implemented. + */ +case class JoinReorderDPFilters(conf: SQLConf) extends PredicateHelper { + /** + * Builds join graph information to be used by the filtering strategies. + * Currently, it builds the sets of star/non-star joins. + * It can be extended with the sets of connected/unconnected joins, which + * can be used to filter Cartesian products. + */ + def buildJoinGraphInfo( + items: Seq[LogicalPlan], + conditions: Set[Expression], + planIndex: Seq[(LogicalPlan, Int)]): Option[JoinGraphInfo] = { + +// Compute the tables in a star-schema relationship. +val starJoin = StarSchemaDetection(conf).findStarJoins(items, conditions.toSeq) +val nonStarJoin = items.filterNot(starJoin.contains(_)) + +if (starJoin.nonEmpty && nonStarJoin.nonEmpty) { + val (starInt, nonStarInt) = planIndex.collect { +case (p, i) if starJoin.contains(p) => + (Some(i), None) +case (p, i) if nonStarJoin.contains(p) => + (None, Some(i)) +case _ => + (None, None) + }.unzip + Some(JoinGraphInfo(starInt.flatten.toSet, nonStarInt.flatten.toSet)) +} else { + // Nothing interesting to return. + None +} + } + + /** + * Applies star-join filter. + * + * Given the outer/inner and the star/non-star sets, + * the following plan combinations are allowed: + * 1) (outer U inner) is a subset of star-join + * 2) star-join is a subset of (outer U inner) + * 3) (outer U inner) is a subset of non star-join + * + * It assumes the sets are disjoint. + * + * Example query graph: + * + * t1 d1 - t2 - t3 + * \ / + * f1 + * | + * d2 + * + * star: {d1, f1, d2} + * non-star: {t2, t1, t3} + * + * level 0: (f1 ), (d2 ), (t3 ), (d1 ), (t1 ), (t2 ) + * level 1: {t3 t2 }, {f1 d2 }, {f1 d1 } + * level 2: {d2 f1 d1 } + * level 3: {t1 d1 f1 d2 }, {t2 d1 f1 d2 } + * level 4: {d1 t2 f1 t1 d2 }, {d1 t3 t2 f1 d2 } + * level 5: {d1 t3 t2 f1 t1 d2 } + */ + def starJoinFilter( + outer: Set[Int], + inner: Set[Int], + filters: JoinGraphInfo) : Boolean = { +val starJoins = filters.starJoins +val nonStarJoins = filters.nonStarJoins +val join = outer.union(inner) + +// Disjoint sets +outer.intersect(inner).isEmpty && + // Either star or non-star is empty + (starJoins.isEmpty || nonStarJoins.isEmpty || +// Join is a subset of the star-join +join.subsetOf(starJoins) || +// Star-join is a subset of join +starJoins.subsetOf(join) || +// Join is a subset of non-star +join.subsetOf(nonStarJoins)) + } +} + +/** + * Helper class that keeps information about the join graph as sets of Int. --- End diff -- sets of Int -> sets of item ids --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user wzhfy commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110089241 --- Diff: sql/catalyst/src/test/scala/org/apache/spark/sql/catalyst/optimizer/StarJoinCostBasedReorderSuite.scala --- @@ -0,0 +1,426 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import org.apache.spark.sql.catalyst.dsl.expressions._ +import org.apache.spark.sql.catalyst.dsl.plans._ +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap} +import org.apache.spark.sql.catalyst.plans.{Inner, PlanTest} +import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, LogicalPlan} +import org.apache.spark.sql.catalyst.rules.RuleExecutor +import org.apache.spark.sql.catalyst.statsEstimation.{StatsEstimationTestBase, StatsTestPlan} +import org.apache.spark.sql.internal.SQLConf +import org.apache.spark.sql.internal.SQLConf._ + + +class StarJoinCostBasedReorderSuite extends PlanTest with StatsEstimationTestBase { + + override val conf = new SQLConf().copy( +CASE_SENSITIVE -> true, +CBO_ENABLED -> true, +JOIN_REORDER_ENABLED -> true, +STARSCHEMA_DETECTION -> true, +JOIN_REORDER_DP_STAR_FILTER -> true) + + object Optimize extends RuleExecutor[LogicalPlan] { +val batches = + Batch("Operator Optimizations", FixedPoint(100), +CombineFilters, +PushDownPredicate, +ReorderJoin(conf), +PushPredicateThroughJoin, +ColumnPruning, +CollapseProject) :: +Batch("Join Reorder", Once, + CostBasedJoinReorder(conf)) :: Nil + } + + private val columnInfo: AttributeMap[ColumnStat] = AttributeMap(Seq( +// F1 (fact table) +attr("f1_fk1") -> ColumnStat(distinctCount = 100, min = Some(1), max = Some(50), --- End diff -- It's an integer attribute, if its value range is [1, 50], ndv can't be larger than 50. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17541 Merged build finished. Test FAILed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17541 **[Test build #75563 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75563/testReport)** for PR 17541 at commit [`3cb7782`](https://github.com/apache/spark/commit/3cb77823e0b6736980ca31f594ff37ac1ddc4242). * This patch **fails Spark unit tests**. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17541 Test FAILed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75563/ Test FAILed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17222: [SPARK-19439][PYSPARK][SQL] PySpark's registerJav...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17222#discussion_r110088421 --- Diff: python/pyspark/sql/context.py --- @@ -228,6 +228,24 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) +@ignore_unicode_prefix +@since(2.2) +def registerJavaUDAF(self, name, javaClassName): +"""Register a java UDAF so it can be used in SQL statements. + +:param name: name of the UDF --- End diff -- name of the UDAF. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17222: [SPARK-19439][PYSPARK][SQL] PySpark's registerJav...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17222#discussion_r110088373 --- Diff: python/pyspark/sql/context.py --- @@ -228,6 +228,24 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) +@ignore_unicode_prefix +@since(2.2) +def registerJavaUDAF(self, name, javaClassName): +"""Register a java UDAF so it can be used in SQL statements. + +:param name: name of the UDF +:param javaClassName: fully qualified name of java class + +>>> sqlContext.registerJavaUDAF("javaUDAF", +... "org.apache.spark.sql.hive.aggregate.MyDoubleAvg") +>>> df = sqlContext.createDataFrame([(1, "a"),(2, "b"), (3, "a")],["id", "name"]) +>>> df.registerTempTable("df") +>>> sqlContext.sql("SELECT name,javaUDAF(id) as avg from df group by name").collect() --- End diff -- nit: name,javaUDAF(id) -> name, javaUDAF(id) --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17222: [SPARK-19439][PYSPARK][SQL] PySpark's registerJav...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17222#discussion_r110088123 --- Diff: python/pyspark/sql/context.py --- @@ -228,6 +228,24 @@ def registerJavaFunction(self, name, javaClassName, returnType=None): jdt = self.sparkSession._jsparkSession.parseDataType(returnType.json()) self.sparkSession._jsparkSession.udf().registerJava(name, javaClassName, jdt) +@ignore_unicode_prefix +@since(2.2) +def registerJavaUDAF(self, name, javaClassName): +"""Register a java UDAF so it can be used in SQL statements. + +:param name: name of the UDF +:param javaClassName: fully qualified name of java class + +>>> sqlContext.registerJavaUDAF("javaUDAF", +... "org.apache.spark.sql.hive.aggregate.MyDoubleAvg") +>>> df = sqlContext.createDataFrame([(1, "a"),(2, "b"), (3, "a")],["id", "name"]) +>>> df.registerTempTable("df") +>>> sqlContext.sql("SELECT name,javaUDAF(id) as avg from df group by name").collect() +[Row(name=u'b', avg=102.0), Row(name=u'a', avg=102.0)] --- End diff -- Shall we add a test failing the registration? E.g. an incorrect class name. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17546: [SPARK-20233] [SQL] Apply star-join filter heuristics to...
Github user wzhfy commented on the issue: https://github.com/apache/spark/pull/17546 So now we have two entry points for star schema, one is in `ReorderJoin` and the other is in `CostBasedJoinReorder`? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17546: [SPARK-20233] [SQL] Apply star-join filter heuristics to...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17546 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17546: [SPARK-20233] [SQL] Apply star-join filter heuristics to...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17546 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75561/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17546: [SPARK-20233] [SQL] Apply star-join filter heuristics to...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17546 **[Test build #75561 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75561/testReport)** for PR 17546 at commit [`5eb9b30`](https://github.com/apache/spark/commit/5eb9b30d4d3ae4d6bc4d137af5ec09a876df805a). * This patch passes all tests. * This patch merges cleanly. * This patch adds the following public classes _(experimental)_: * `case class JoinReorderDP(conf: SQLConf) extends PredicateHelper with Logging ` * `case class JoinReorderDPFilters(conf: SQLConf) extends PredicateHelper ` * `case class JoinGraphInfo (starJoins: Set[Int], nonStarJoins: Set[Int])` --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17505: [SPARK-20187][SQL] Replace loadTable with moveFil...
Github user wangyum closed the pull request at: https://github.com/apache/spark/pull/17505 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17505: [SPARK-20187][SQL] Replace loadTable with moveFile to sp...
Github user wangyum commented on the issue: https://github.com/apache/spark/pull/17505 @gatorsmile You are right. It is similar to [HIVE-12908](https://github.com/apache/hive/commit/26268deb4844d3f3c530769c6276b17b0c6caaa0). There are 3 bottlenecks for many output files: 1. Set `spark.hadoop.mapreduce.fileoutputcommitter.algorithm.version=2` to speed up HadoopMapReduceCommitProtocol.commitJob for many output files, more see: [SPARK-20107](https://issues.apache.org/jira/browse/SPARK-20107). 2. Hive.loadTable load files one by one, it can be fixed by [HIVE-12908](https://issues.apache.org/jira/browse/HIVE-12908). 3. [ShimLoader.getHadoopShims().setFullFileStatus(conf, destStatus, fs, destf)](https://github.com/apache/hive/blob/release-1.2.1/ql/src/java/org/apache/hadoop/hive/ql/metadata/Hive.java#L2649) very slow for many files, It seems to be difficult to optimize. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17541 **[Test build #75563 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75563/testReport)** for PR 17541 at commit [`3cb7782`](https://github.com/apache/spark/commit/3cb77823e0b6736980ca31f594ff37ac1ddc4242). --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/17541#discussion_r110086440 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala --- @@ -423,8 +423,17 @@ case class CatalogRelation( Objects.hashCode(tableMeta.identifier, output) } - /** Only compare table identifier. */ - override lazy val cleanArgs: Seq[Any] = Seq(tableMeta.identifier) + override lazy val canonicalized: LogicalPlan = { +CatalogRelation( + /** Only care about table identifier. */ + tableMeta = CatalogTable( +tableMeta.identifier, +CatalogTableType.MANAGED, +CatalogStorageFormat.empty, +new StructType()), + dataCols = dataCols.map(_.canonicalized.asInstanceOf[Attribute]), + partitionCols = partitionCols.map(_.canonicalized.asInstanceOf[Attribute])) --- End diff -- `CatalogRelation` has 3 asserts at the beginning, so we can't simply use Nil --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user cloud-fan commented on the issue: https://github.com/apache/spark/pull/17541 retest this please --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/17541#discussion_r110086068 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/plans/physical/broadcastMode.scala --- @@ -26,10 +26,7 @@ import org.apache.spark.sql.catalyst.InternalRow trait BroadcastMode { def transform(rows: Array[InternalRow]): Any - /** - * Returns true iff this [[BroadcastMode]] generates the same result as `other`. - */ - def compatibleWith(other: BroadcastMode): Boolean --- End diff -- `BroadcastMode` is a field of `BroadcastExchangeExec`. Since we need to canonicalize a `QueryPlan`, the `BroadcastMode` also need to be canonicalized. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110083521 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala --- @@ -327,3 +345,104 @@ object JoinReorderDP extends PredicateHelper with Logging { case class Cost(card: BigInt, size: BigInt) { def +(other: Cost): Cost = Cost(this.card + other.card, this.size + other.size) } + +/** + * Implements optional filters to reduce the search space for join enumeration. + * + * 1) Star-join filters: Plan star-joins together since they are assumed + *to have an optimal execution based on their RI relationship. + * 2) Cartesian products: Defer their planning later in the graph to avoid + *large intermediate results (expanding joins, in general). + * 3) Composite inners: Don't generate "bushy tree" plans to avoid materializing + * intermediate results. + * + * Filters (2) and (3) are not implemented. + */ +case class JoinReorderDPFilters(conf: SQLConf) extends PredicateHelper { + /** + * Builds join graph information to be used by the filtering strategies. + * Currently, it builds the sets of star/non-star joins. + * It can be extended with the sets of connected/unconnected joins, which + * can be used to filter Cartesian products. + */ + def buildJoinGraphInfo( + items: Seq[LogicalPlan], + conditions: Set[Expression], + planIndex: Seq[(LogicalPlan, Int)]): Option[JoinGraphInfo] = { + +// Compute the tables in a star-schema relationship. +val starJoin = StarSchemaDetection(conf).findStarJoins(items, conditions.toSeq) +val nonStarJoin = items.filterNot(starJoin.contains(_)) + +if (starJoin.nonEmpty && nonStarJoin.nonEmpty) { + val (starInt, nonStarInt) = planIndex.collect { +case (p, i) if starJoin.contains(p) => + (Some(i), None) +case (p, i) if nonStarJoin.contains(p) => + (None, Some(i)) +case _ => + (None, None) + }.unzip + Some(JoinGraphInfo(starInt.flatten.toSet, nonStarInt.flatten.toSet)) +} else { + // Nothing interesting to return. + None +} + } + + /** + * Applies star-join filter. + * + * Given the outer/inner and the star/non-star sets, + * the following plan combinations are allowed: + * 1) (outer U inner) is a subset of star-join + * 2) star-join is a subset of (outer U inner) + * 3) (outer U inner) is a subset of non star-join + * + * It assumes the sets are disjoint. + * + * Example query graph: + * + * t1 d1 - t2 - t3 + * \ / + * f1 + * | + * d2 + * + * star: {d1, f1, d2} + * non-star: {t2, t1, t3} + * + * level 0: (f1 ), (d2 ), (t3 ), (d1 ), (t1 ), (t2 ) + * level 1: {t3 t2 }, {f1 d2 }, {f1 d1 } + * level 2: {d2 f1 d1 } + * level 3: {t1 d1 f1 d2 }, {t2 d1 f1 d2 } + * level 4: {d1 t2 f1 t1 d2 }, {d1 t3 t2 f1 d2 } + * level 5: {d1 t3 t2 f1 t1 d2 } + */ + def starJoinFilter( + outer: Set[Int], + inner: Set[Int], + filters: JoinGraphInfo) : Boolean = { +val starJoins = filters.starJoins +val nonStarJoins = filters.nonStarJoins +val join = outer.union(inner) + +// Disjoint sets +outer.intersect(inner).isEmpty && + // Either star or non-star is empty + (starJoins.isEmpty || nonStarJoins.isEmpty || +// Join is a subset of the star-join +join.subsetOf(starJoins) || +// Star-join is a subset of join +starJoins.subsetOf(join) || --- End diff -- Can't a star join within a built join be reordered? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #16906: [SPARK-19570][PYSPARK] Allow to disable hive in pyspark ...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/16906 **[Test build #75562 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75562/testReport)** for PR 16906 at commit [`63942c1`](https://github.com/apache/spark/commit/63942c1ca310cfa8183fc00eb3b6c3cbb4d0926c). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #16906: [SPARK-19570][PYSPARK] Allow to disable hive in pyspark ...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/16906 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75562/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #16906: [SPARK-19570][PYSPARK] Allow to disable hive in pyspark ...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/16906 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17541: [SPARK-20229][SQL] add semanticHash to QueryPlan
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/17541#discussion_r110080643 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/catalog/interface.scala --- @@ -423,8 +423,17 @@ case class CatalogRelation( Objects.hashCode(tableMeta.identifier, output) } - /** Only compare table identifier. */ - override lazy val cleanArgs: Seq[Any] = Seq(tableMeta.identifier) + override lazy val canonicalized: LogicalPlan = { +CatalogRelation( + /** Only care about table identifier. */ + tableMeta = CatalogTable( +tableMeta.identifier, +CatalogTableType.MANAGED, +CatalogStorageFormat.empty, +new StructType()), + dataCols = dataCols.map(_.canonicalized.asInstanceOf[Attribute]), + partitionCols = partitionCols.map(_.canonicalized.asInstanceOf[Attribute])) --- End diff -- We only care about table identifier. How about setting them to Nil ``` dataCols = Nil, partitionCols = Nil ``` --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #16906: [SPARK-19570][PYSPARK] Allow to disable hive in pyspark ...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/16906 **[Test build #75562 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75562/testReport)** for PR 16906 at commit [`63942c1`](https://github.com/apache/spark/commit/63942c1ca310cfa8183fc00eb3b6c3cbb4d0926c). --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17222: [SPARK-19439][PYSPARK][SQL] PySpark's registerJavaFuncti...
Github user zjffdu commented on the issue: https://github.com/apache/spark/pull/17222 @holdenk Mind to review it ? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110078548 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala --- @@ -327,3 +345,104 @@ object JoinReorderDP extends PredicateHelper with Logging { case class Cost(card: BigInt, size: BigInt) { def +(other: Cost): Cost = Cost(this.card + other.card, this.size + other.size) } + +/** + * Implements optional filters to reduce the search space for join enumeration. + * + * 1) Star-join filters: Plan star-joins together since they are assumed + *to have an optimal execution based on their RI relationship. + * 2) Cartesian products: Defer their planning later in the graph to avoid + *large intermediate results (expanding joins, in general). + * 3) Composite inners: Don't generate "bushy tree" plans to avoid materializing + * intermediate results. + * + * Filters (2) and (3) are not implemented. + */ +case class JoinReorderDPFilters(conf: SQLConf) extends PredicateHelper { + /** + * Builds join graph information to be used by the filtering strategies. + * Currently, it builds the sets of star/non-star joins. + * It can be extended with the sets of connected/unconnected joins, which + * can be used to filter Cartesian products. + */ + def buildJoinGraphInfo( + items: Seq[LogicalPlan], + conditions: Set[Expression], + planIndex: Seq[(LogicalPlan, Int)]): Option[JoinGraphInfo] = { + +// Compute the tables in a star-schema relationship. +val starJoin = StarSchemaDetection(conf).findStarJoins(items, conditions.toSeq) +val nonStarJoin = items.filterNot(starJoin.contains(_)) + +if (starJoin.nonEmpty && nonStarJoin.nonEmpty) { + val (starInt, nonStarInt) = planIndex.collect { +case (p, i) if starJoin.contains(p) => + (Some(i), None) +case (p, i) if nonStarJoin.contains(p) => + (None, Some(i)) +case _ => + (None, None) + }.unzip + Some(JoinGraphInfo(starInt.flatten.toSet, nonStarInt.flatten.toSet)) +} else { + // Nothing interesting to return. + None +} + } + + /** + * Applies star-join filter. + * + * Given the outer/inner and the star/non-star sets, + * the following plan combinations are allowed: + * 1) (outer U inner) is a subset of star-join + * 2) star-join is a subset of (outer U inner) + * 3) (outer U inner) is a subset of non star-join + * + * It assumes the sets are disjoint. + * + * Example query graph: + * + * t1 d1 - t2 - t3 + * \ / + * f1 + * | + * d2 + * + * star: {d1, f1, d2} + * non-star: {t2, t1, t3} + * + * level 0: (f1 ), (d2 ), (t3 ), (d1 ), (t1 ), (t2 ) + * level 1: {t3 t2 }, {f1 d2 }, {f1 d1 } + * level 2: {d2 f1 d1 } + * level 3: {t1 d1 f1 d2 }, {t2 d1 f1 d2 } + * level 4: {d1 t2 f1 t1 d2 }, {d1 t3 t2 f1 d2 } + * level 5: {d1 t3 t2 f1 t1 d2 } + */ + def starJoinFilter( + outer: Set[Int], + inner: Set[Int], + filters: JoinGraphInfo) : Boolean = { +val starJoins = filters.starJoins +val nonStarJoins = filters.nonStarJoins +val join = outer.union(inner) + +// Disjoint sets +outer.intersect(inner).isEmpty && --- End diff -- I think the check in the beginning of `buildJoin` can guarantee this already. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17050: [SPARK-19722] [SQL] [MINOR] Clean up the usage of...
Github user gatorsmile closed the pull request at: https://github.com/apache/spark/pull/17050 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17546: [SPARK-20233] [SQL] Apply star-join filter heuristics to...
Github user ioana-delaney commented on the issue: https://github.com/apache/spark/pull/17546 @wzhfy @gatorsmile @cloud-fan I've integrated star-join with join enumeration. Would you please take a look? Thanks. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
Github user ioana-delaney commented on a diff in the pull request: https://github.com/apache/spark/pull/17546#discussion_r110076651 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/CostBasedJoinReorder.scala --- @@ -150,12 +148,15 @@ object JoinReorderDP extends PredicateHelper with Logging { case (item, id) => Set(id) -> JoinPlan(Set(id), item, Set(), Cost(0, 0)) }.toMap) +// Build filters from the join graph to be used by the search algorithm. +val filters = JoinReorderDPFilters(conf).buildJoinGraphInfo(items, conditions, itemIndex) + // Build plans for next levels until the last level has only one plan. This plan contains // all items that can be joined, so there's no need to continue. val topOutputSet = AttributeSet(output) -while (foundPlans.size < items.length && foundPlans.last.size > 1) { +while (foundPlans.size < items.length) { --- End diff -- @wzhfy Condition "foundPlans.last.size > 1" does not apply when filters are used with the join enumeration since not all the plan combinations are generated. Without filters, I think the condition will always be satisfied, so I removed it completely. Let me know if you have a counter example. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17546: [SPARK-20233] [SQL] Apply star-join filter heuristics to...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17546 **[Test build #75561 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75561/testReport)** for PR 17546 at commit [`5eb9b30`](https://github.com/apache/spark/commit/5eb9b30d4d3ae4d6bc4d137af5ec09a876df805a). --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17546: [SPARK-20233] [SQL] Apply star-join filter heuris...
GitHub user ioana-delaney opened a pull request: https://github.com/apache/spark/pull/17546 [SPARK-20233] [SQL] Apply star-join filter heuristics to dynamic programming join enumeration ## What changes were proposed in this pull request? Implements star-join filter to reduce the search space for dynamic programming join enumeration. Consider the following join graph: ``` T1 D1 - T2 - T3 \ / F1 | D2 star-join: {F1, D1, D2} non-star: {T1, T2, T3} ``` The following join combinations will be generated: ``` level 0: (F1), (D1), (D2), (T1), (T2), (T3) level 1: {F1, D1}, {F1, D2}, {T2, T3} level 2: {F1, D1, D2} level 3: {F1, D1, D2, T1}, {F1, D1, D2, T2} level 4: {F1, D1, D2, T1, T2}, {F1, D1, D2, T2, T3 } level 6: {F1, D1, D2, T1, T2, T3} ``` ## How was this patch tested? New test suite ```StarJOinCostBasedReorderSuite.scala```. You can merge this pull request into a Git repository by running: $ git pull https://github.com/ioana-delaney/spark starSchemaCBOv3 Alternatively you can review and apply these changes as the patch at: https://github.com/apache/spark/pull/17546.patch To close this pull request, make a commit to your master/trunk branch with (at least) the following in the commit message: This closes #17546 commit 5eb9b30d4d3ae4d6bc4d137af5ec09a876df805a Author: Ioana Delaney Date: 2017-04-06T04:04:20Z [SPARK-20233] Implement star-join filter for join enumeration. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17527: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String t...
Github user HyukjinKwon commented on the issue: https://github.com/apache/spark/pull/17527 Yea, that's the concern. The downside is when these are exposed to users. However, it might be an advantage as well. The behavior doesn't depend on default JVM locale and is consistent. I think the argument is the same with [SPARK-18076](https://issues.apache.org/jira/browse/SPARK-18076). If it is easy to leave out the exposed ones, I think that's also an option. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17527: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String t...
Github user viirya commented on the issue: https://github.com/apache/spark/pull/17527 @HyukjinKwon yap, so for such cases exposed to users, I think it is better to leave it out for the default locale? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17527: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String t...
Github user HyukjinKwon commented on the issue: https://github.com/apache/spark/pull/17527 @viirya, I think it is possible. In case of `Lower`, `Upper` and `InitCap` as an example maybe. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17527: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java S...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/17527#discussion_r110064613 --- Diff: common/unsafe/src/main/java/org/apache/spark/unsafe/types/UTF8String.java --- @@ -407,7 +408,7 @@ public UTF8String toLowerCase() { } private UTF8String toLowerCaseSlow() { -return fromString(toString().toLowerCase()); +return fromString(toString().toLowerCase(Locale.ROOT)); --- End diff -- It seems there are few cases exposed to users. For example, this seems used in `Lower` and `InitCap` expressions, where, up to my knowledge, the lower cased ones are exposed into users. In this case, IIRC the same argument (consistent base + options for variants approach) is applied in [SPARK-18076](https://issues.apache.org/jira/browse/SPARK-18076). So, I think it might be fine. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17527: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String t...
Github user viirya commented on the issue: https://github.com/apache/spark/pull/17527 Out of curious, is there any situation we do really need the locale setting, instead of `Locale.ROOT`? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #16793: [SPARK-19454][PYTHON][SQL] DataFrame.replace improvement...
Github user zero323 commented on the issue: https://github.com/apache/spark/pull/16793 Thanks @holdenk --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17531: [SPARK-20217][core] Executor should not fail stag...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/17531 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17535: [SPARK-20222][SQL] Bring back the Spark SQL UI when exec...
Github user carsonwang commented on the issue: https://github.com/apache/spark/pull/17535 Yes, it is closely related but two scenarios of adding `SQLExecution.withNewExecutionId`. Now some tests fail because `withNewExecutionId` is called twice. ``` java.lang.IllegalArgumentException: spark.sql.execution.id is already set [info] at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:90) [info] at org.apache.spark.sql.execution.datasources.FileFormatWriter$.write(FileFormatWriter.scala:164) [info] at org.apache.spark.sql.hive.execution.InsertIntoHiveTable.run(InsertIntoHiveTable.scala:311) [info] at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult$lzycompute(commands.scala:58) [info] at org.apache.spark.sql.execution.command.ExecutedCommandExec.sideEffectResult(commands.scala:56) [info] at org.apache.spark.sql.execution.command.ExecutedCommandExec.executeCollect(commands.scala:67) [info] at org.apache.spark.sql.execution.SparkPlan.executeCollectPublic(SparkPlan.scala:295) [info] at org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$3.apply(QueryExecution.scala:137) [info] at org.apache.spark.sql.execution.QueryExecution$$anonfun$hiveResultString$3.apply(QueryExecution.scala:136) [info] at org.apache.spark.sql.execution.SQLExecution$.withNewExecutionId(SQLExecution.scala:65) [info] at org.apache.spark.sql.execution.QueryExecution.hiveResultString(QueryExecution.scala:136) ``` --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user dilipbiswal commented on the issue: https://github.com/apache/spark/pull/17537 @viirya @cloud-fan @gatorsmile Thanks a lot. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17544: [SPARK-20231] [SQL] Refactor star schema code for...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/17544 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17544: [SPARK-20231] [SQL] Refactor star schema code for the su...
Github user ioana-delaney commented on the issue: https://github.com/apache/spark/pull/17544 @gatorsmile Thank you! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17538: [SPARK-20223][SQL] Fix typo in tpcds q77.sql
Github user wzhfy commented on the issue: https://github.com/apache/spark/pull/17538 @srowen It's not run automatically. And even if it is run, no error would be caught, the predicate will just become `null` because casting `'2000-08-03]'` to Date is failed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17544: [SPARK-20231] [SQL] Refactor star schema code for the su...
Github user gatorsmile commented on the issue: https://github.com/apache/spark/pull/17544 Thanks! Merging to master. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17544: [SPARK-20231] [SQL] Refactor star schema code for the su...
Github user gatorsmile commented on the issue: https://github.com/apache/spark/pull/17544 LGTM. Look forward to your next PR for merging start join detection with CBO. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17544: [SPARK-20231] [SQL] Refactor star schema code for...
Github user gatorsmile commented on a diff in the pull request: https://github.com/apache/spark/pull/17544#discussion_r110060504 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/StarSchemaDetection.scala --- @@ -0,0 +1,351 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + *http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.spark.sql.catalyst.optimizer + +import scala.annotation.tailrec + +import org.apache.spark.sql.catalyst.expressions._ +import org.apache.spark.sql.catalyst.planning.PhysicalOperation +import org.apache.spark.sql.catalyst.plans._ +import org.apache.spark.sql.catalyst.plans.logical._ +import org.apache.spark.sql.internal.SQLConf + +/** + * Encapsulates star-schema detection logic. + */ +case class StarSchemaDetection(conf: SQLConf) extends PredicateHelper { + + /** + * Star schema consists of one or more fact tables referencing a number of dimension + * tables. In general, star-schema joins are detected using the following conditions: + * 1. Informational RI constraints (reliable detection) + * + Dimension contains a primary key that is being joined to the fact table. + * + Fact table contains foreign keys referencing multiple dimension tables. + * 2. Cardinality based heuristics + * + Usually, the table with the highest cardinality is the fact table. + * + Table being joined with the most number of tables is the fact table. + * + * To detect star joins, the algorithm uses a combination of the above two conditions. + * The fact table is chosen based on the cardinality heuristics, and the dimension + * tables are chosen based on the RI constraints. A star join will consist of the largest + * fact table joined with the dimension tables on their primary keys. To detect that a + * column is a primary key, the algorithm uses table and column statistics. + * + * The algorithm currently returns only the star join with the largest fact table. + * Choosing the largest fact table on the driving arm to avoid large inners is in + * general a good heuristic. This restriction will be lifted to observe multiple + * star joins. + * + * The highlights of the algorithm are the following: + * + * Given a set of joined tables/plans, the algorithm first verifies if they are eligible + * for star join detection. An eligible plan is a base table access with valid statistics. + * A base table access represents Project or Filter operators above a LeafNode. Conservatively, + * the algorithm only considers base table access as part of a star join since they provide + * reliable statistics. This restriction can be lifted with the CBO enablement by default. + * + * If some of the plans are not base table access, or statistics are not available, the algorithm + * returns an empty star join plan since, in the absence of statistics, it cannot make + * good planning decisions. Otherwise, the algorithm finds the table with the largest cardinality + * (number of rows), which is assumed to be a fact table. + * + * Next, it computes the set of dimension tables for the current fact table. A dimension table + * is assumed to be in a RI relationship with a fact table. To infer column uniqueness, + * the algorithm compares the number of distinct values with the total number of rows in the + * table. If their relative difference is within certain limits (i.e. ndvMaxError * 2, adjusted + * based on 1TB TPC-DS data), the column is assumed to be unique. + */ + def findStarJoins( + input: Seq[LogicalPlan], + conditions: Seq[Expression]): Seq[LogicalPlan] = { + +val emptyStarJoinPlan = Seq.empty[LogicalPlan] + +if (!conf.starSchemaDetection || input.size < 2) { + emptyStarJoinPlan +} else { + // Find if the input plans are eligible for star join detection. + // An eligible plan is a base table acces
[GitHub] spark pull request #17149: [SPARK-19257][SQL]location for table/partition/da...
Github user HyukjinKwon commented on a diff in the pull request: https://github.com/apache/spark/pull/17149#discussion_r110060244 --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala --- @@ -285,7 +285,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // compatible format, which means the data source is file-based and must have a `path`. require(table.storage.locationUri.isDefined, "External file-based data source table must have a `path` entry in storage properties.") -Some(new Path(table.location).toUri.toString) --- End diff -- Thank you for confirming. Let me send a mail soon. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17506: [SPARK-20189][DStream] Fix spark kinesis testcases to re...
Github user yssharma commented on the issue: https://github.com/apache/spark/pull/17506 @srowen - does the Jenkins re-test trigger automatically? else, could I request a retest on this patch please ? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17532: [SPARK-20214][ML] Make sure converted csc matrix ...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/17532 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user jkbradley commented on the issue: https://github.com/apache/spark/pull/17532 LGTM Merging with master and branch-2.1, branch-2.0 Thanks a lot! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17149: [SPARK-19257][SQL]location for table/partition/da...
Github user cloud-fan commented on a diff in the pull request: https://github.com/apache/spark/pull/17149#discussion_r110058436 --- Diff: sql/hive/src/main/scala/org/apache/spark/sql/hive/HiveExternalCatalog.scala --- @@ -285,7 +285,7 @@ private[spark] class HiveExternalCatalog(conf: SparkConf, hadoopConf: Configurat // compatible format, which means the data source is file-based and must have a `path`. require(table.storage.locationUri.isDefined, "External file-based data source table must have a `path` entry in storage properties.") -Some(new Path(table.location).toUri.toString) --- End diff -- yea, please go ahead --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17149: [SPARK-19257][SQL]location for table/partition/database ...
Github user cloud-fan commented on the issue: https://github.com/apache/spark/pull/17149 yea, please go ahead --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17537: [SPARK-20204][SQL][Followup] SQLConf should react...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/17537 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user cloud-fan commented on the issue: https://github.com/apache/spark/pull/17537 thanks, merging to master! --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17532: [SPARK-20214][ML] Make sure converted csc matrix ...
Github user jkbradley commented on a diff in the pull request: https://github.com/apache/spark/pull/17532#discussion_r110057924 --- Diff: python/pyspark/mllib/tests.py --- @@ -853,6 +853,17 @@ def serialize(l): self.assertEqual(sv, serialize(lil.tocsr())) self.assertEqual(sv, serialize(lil.todok())) +def test_convert_to_vector(self): --- End diff -- No it's Ok; I made a separate JIRA to port the tests there. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user yhuai commented on the issue: https://github.com/apache/spark/pull/17531 Thanks. Merging to master. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17531 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17531 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75559/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17531 **[Test build #75559 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75559/testReport)** for PR 17531 at commit [`42f49f2`](https://github.com/apache/spark/commit/42f49f21f20e82cc64d346b163523a9d17ae7bf8). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user viirya commented on the issue: https://github.com/apache/spark/pull/17537 LGTM --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17296: [SPARK-19953][ML] Random Forest Models use parent UID wh...
Github user BryanCutler commented on the issue: https://github.com/apache/spark/pull/17296 @MLnick this should be good to go. I made https://issues.apache.org/jira/browse/SPARK-20234 to address some better consistency in these basic checks. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17532 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17532 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75560/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17532 **[Test build #75560 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75560/testReport)** for PR 17532 at commit [`2612d66`](https://github.com/apache/spark/commit/2612d66c9eb5ebdf1ca5dfb61a687e37afbcb54a). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17537 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17537 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75558/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17537 **[Test build #75558 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75558/testReport)** for PR 17537 at commit [`1fb23cf`](https://github.com/apache/spark/commit/1fb23cf8417bdb0da7c01c7e646b7e16c1fd511a). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17532: [SPARK-20214][ML] Make sure converted csc matrix ...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17532#discussion_r110053944 --- Diff: python/pyspark/mllib/tests.py --- @@ -853,6 +853,17 @@ def serialize(l): self.assertEqual(sv, serialize(lil.tocsr())) self.assertEqual(sv, serialize(lil.todok())) +def test_convert_to_vector(self): --- End diff -- We have no Scipy-related tests in `ml` submodule, so I don't add it. If we really need one, please let me know. I can add it quickly. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17527: [SPARK-20156][CORE][SQL][STREAMING][MLLIB] Java String t...
Github user HyukjinKwon commented on the issue: https://github.com/apache/spark/pull/17527 I support this idea in general. I can at least identify several references, for example,, https://hibernate.atlassian.net/plugins/servlet/mobile#issue/HHH-9722, `https://github.com/hibernate/hibernate-orm/pull/931` and https://www.google.co.kr/amp/s/garygregory.wordpress.com/2015/11/03/java-lowercase-conversion-turkey/amp/ Let me investigate possible downside within today. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17494: [SPARK-20076][ML][PySpark] Add Python interface for ml.s...
Github user viirya commented on the issue: https://github.com/apache/spark/pull/17494 @jkbradley @MLnick @holdenk If there is no more questions about this change, maybe we can make it into 2.2? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17532 **[Test build #75560 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75560/testReport)** for PR 17532 at commit [`2612d66`](https://github.com/apache/spark/commit/2612d66c9eb5ebdf1ca5dfb61a687e37afbcb54a). --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user viirya commented on the issue: https://github.com/apache/spark/pull/17532 @jkbradley An unit test is added. Please check this again. Thanks. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17532: [SPARK-20214][ML] Make sure converted csc matrix ...
Github user viirya commented on a diff in the pull request: https://github.com/apache/spark/pull/17532#discussion_r110052585 --- Diff: python/pyspark/ml/linalg/__init__.py --- @@ -72,7 +72,9 @@ def _convert_to_vector(l): return DenseVector(l) elif _have_scipy and scipy.sparse.issparse(l): assert l.shape[1] == 1, "Expected column vector" -csc = l.tocsc() +# Make sure the converted csc_matrix has sorted indices. +csc = l.tocsr().tocsc() --- End diff -- Thanks. Fixed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17532: [SPARK-20214][ML] Make sure converted csc matrix has sor...
Github user viirya commented on the issue: https://github.com/apache/spark/pull/17532 @jkbradley Thanks for comment. I will add the unit test now. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17544: [SPARK-20231] [SQL] Refactor star schema code for the su...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17544 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17544: [SPARK-20231] [SQL] Refactor star schema code for the su...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17544 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75557/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17544: [SPARK-20231] [SQL] Refactor star schema code for the su...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17544 **[Test build #75557 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75557/testReport)** for PR 17544 at commit [`99732ff`](https://github.com/apache/spark/commit/99732ff5b76c83faeb95b1a81d497d0697c32d3e). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17539: [SPARK-20224][SS] Updated docs for streaming drop...
Github user asfgit closed the pull request at: https://github.com/apache/spark/pull/17539 --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17530: [SPARK-5158] Access kerberized HDFS from Spark standalon...
Github user vanzin commented on the issue: https://github.com/apache/spark/pull/17530 > That would work for cluster mode but in client mode the driver on the submitting nodes still needs the keytab unfortunately. You're setting up a special cluster for a single user. I'm pretty sure that user can login to the KDC on the driver host. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17530: [SPARK-5158] Access kerberized HDFS from Spark standalon...
Github user vanzin commented on the issue: https://github.com/apache/spark/pull/17530 I'm just not sold on the idea that this is necessary in the first place. Personally I don't use standalone nor do I play with it at all, so my concerns are purely from a security standpoint. As in, if I were managing a secure cluster, why would I ever allow this code to run on it? If you can write a spec and convince people who maintain the standalone manager that it's a good idea, then go for it. I just want to make sure you're not setting up people to completely break security on their clusters. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17530: [SPARK-5158] Access kerberized HDFS from Spark standalon...
Github user themodernlife commented on the issue: https://github.com/apache/spark/pull/17530 BTW not trying to give you the hard sell and appreciate the help rounding out the requirements from the core committers' POV. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17530: [SPARK-5158] Access kerberized HDFS from Spark standalon...
Github user themodernlife commented on the issue: https://github.com/apache/spark/pull/17530 That would work for cluster mode but in client mode the driver on the submitting nodes still needs the keytab unfortunately. Standalone clusters are best viewed as distributed single-user programs, so I think the real mistake is not bringing them into a secure environment, but bringing them into a secure environment and trying to use them in a multi-tenant/multi-user fashion. I can see the concern that this feature might give someone who brings standalone clusters into a kerberized environment a false sense of security. What about disabling unless something like `spark.standalone.single-user` is set to true? --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17530: [SPARK-5158] Access kerberized HDFS from Spark standalon...
Github user vanzin commented on the issue: https://github.com/apache/spark/pull/17530 And BTW, if you really want to pursue this, please write a detailed spec explaining everything that is being done, and describe all the security issues people need to be aware of. It might even be prudent to make it an SPIP. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17543: [SPARK-20230] FetchFailedExceptions should invalidate fi...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17543 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75554/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17543: [SPARK-20230] FetchFailedExceptions should invalidate fi...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17543 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17543: [SPARK-20230] FetchFailedExceptions should invalidate fi...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17543 **[Test build #75554 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75554/testReport)** for PR 17543 at commit [`bc80aab`](https://github.com/apache/spark/commit/bc80aab919cabe93c59a312e7e72ed1ed453906d). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17539: [SPARK-20224][SS] Updated docs for streaming dropDuplica...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17539 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17539: [SPARK-20224][SS] Updated docs for streaming dropDuplica...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17539 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/75556/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17539: [SPARK-20224][SS] Updated docs for streaming dropDuplica...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17539 **[Test build #75556 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75556/testReport)** for PR 17539 at commit [`c78ebe8`](https://github.com/apache/spark/commit/c78ebe8b466717f5e31d4ab39e48ad7b358b59a0). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17512: [SPARK-20196][PYTHON][SQL] update doc for catalog functi...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17512 Merged build finished. Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17512: [SPARK-20196][PYTHON][SQL] update doc for catalog functi...
Github user AmplabJenkins commented on the issue: https://github.com/apache/spark/pull/17512 Test PASSed. Refer to this link for build results (access rights to CI server needed): https://amplab.cs.berkeley.edu/jenkins//job/SparkPullRequestBuilder/7/ Test PASSed. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17512: [SPARK-20196][PYTHON][SQL] update doc for catalog functi...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17512 **[Test build #7 has finished](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/7/testReport)** for PR 17512 at commit [`5ed1950`](https://github.com/apache/spark/commit/5ed195013d215340e6a609d01f1314e6454ac4e9). * This patch passes all tests. * This patch merges cleanly. * This patch adds no public classes. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user JoshRosen commented on the issue: https://github.com/apache/spark/pull/17531 LGTM pending Jenkins. --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17537: [SPARK-20204][SQL][Followup] SQLConf should react to cha...
Github user gatorsmile commented on the issue: https://github.com/apache/spark/pull/17537 LGTM pending Jenkins --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user SparkQA commented on the issue: https://github.com/apache/spark/pull/17531 **[Test build #75559 has started](https://amplab.cs.berkeley.edu/jenkins/job/SparkPullRequestBuilder/75559/testReport)** for PR 17531 at commit [`42f49f2`](https://github.com/apache/spark/commit/42f49f21f20e82cc64d346b163523a9d17ae7bf8). --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org
[GitHub] spark pull request #17544: [SPARK-20231] [SQL] Refactor star schema code for...
Github user ioana-delaney commented on a diff in the pull request: https://github.com/apache/spark/pull/17544#discussion_r11001 --- Diff: sql/catalyst/src/main/scala/org/apache/spark/sql/catalyst/optimizer/joins.scala --- @@ -20,339 +20,13 @@ package org.apache.spark.sql.catalyst.optimizer import scala.annotation.tailrec import org.apache.spark.sql.catalyst.expressions._ -import org.apache.spark.sql.catalyst.planning.{ExtractFiltersAndInnerJoins, PhysicalOperation} +import org.apache.spark.sql.catalyst.planning.ExtractFiltersAndInnerJoins import org.apache.spark.sql.catalyst.plans._ import org.apache.spark.sql.catalyst.plans.logical._ import org.apache.spark.sql.catalyst.rules._ import org.apache.spark.sql.internal.SQLConf /** - * Encapsulates star-schema join detection. - */ -case class StarSchemaDetection(conf: SQLConf) extends PredicateHelper { - - /** - * Star schema consists of one or more fact tables referencing a number of dimension - * tables. In general, star-schema joins are detected using the following conditions: - * 1. Informational RI constraints (reliable detection) - *+ Dimension contains a primary key that is being joined to the fact table. - *+ Fact table contains foreign keys referencing multiple dimension tables. - * 2. Cardinality based heuristics - *+ Usually, the table with the highest cardinality is the fact table. - *+ Table being joined with the most number of tables is the fact table. - * - * To detect star joins, the algorithm uses a combination of the above two conditions. - * The fact table is chosen based on the cardinality heuristics, and the dimension - * tables are chosen based on the RI constraints. A star join will consist of the largest - * fact table joined with the dimension tables on their primary keys. To detect that a - * column is a primary key, the algorithm uses table and column statistics. - * - * Since Catalyst only supports left-deep tree plans, the algorithm currently returns only - * the star join with the largest fact table. Choosing the largest fact table on the - * driving arm to avoid large inners is in general a good heuristic. This restriction can - * be lifted with support for bushy tree plans. - * - * The highlights of the algorithm are the following: - * - * Given a set of joined tables/plans, the algorithm first verifies if they are eligible - * for star join detection. An eligible plan is a base table access with valid statistics. - * A base table access represents Project or Filter operators above a LeafNode. Conservatively, - * the algorithm only considers base table access as part of a star join since they provide - * reliable statistics. - * - * If some of the plans are not base table access, or statistics are not available, the algorithm - * returns an empty star join plan since, in the absence of statistics, it cannot make - * good planning decisions. Otherwise, the algorithm finds the table with the largest cardinality - * (number of rows), which is assumed to be a fact table. - * - * Next, it computes the set of dimension tables for the current fact table. A dimension table - * is assumed to be in a RI relationship with a fact table. To infer column uniqueness, - * the algorithm compares the number of distinct values with the total number of rows in the - * table. If their relative difference is within certain limits (i.e. ndvMaxError * 2, adjusted - * based on 1TB TPC-DS data), the column is assumed to be unique. - */ - def findStarJoins( - input: Seq[LogicalPlan], - conditions: Seq[Expression]): Seq[Seq[LogicalPlan]] = { - -val emptyStarJoinPlan = Seq.empty[Seq[LogicalPlan]] - -if (!conf.starSchemaDetection || input.size < 2) { - emptyStarJoinPlan -} else { - // Find if the input plans are eligible for star join detection. - // An eligible plan is a base table access with valid statistics. - val foundEligibleJoin = input.forall { -case PhysicalOperation(_, _, t: LeafNode) if t.stats(conf).rowCount.isDefined => true -case _ => false - } - - if (!foundEligibleJoin) { -// Some plans don't have stats or are complex plans. Conservatively, -// return an empty star join. This restriction can be lifted -// once statistics are propagated in the plan. -emptyStarJoinPlan - } else { -// Find the fact table using cardinality based heuristics i.e. -// the table with the largest number of rows. -val sortedFactTables = input.map { plan => - T
[GitHub] spark issue #17531: [SPARK-20217][core] Executor should not fail stage if ki...
Github user yhuai commented on the issue: https://github.com/apache/spark/pull/17531 test this please --- If your project is set up for it, you can reply to this email and have your reply appear on GitHub as well. If your project does not have this feature enabled and wishes so, or if the feature is enabled but not working, please contact infrastructure at infrastruct...@apache.org or file a JIRA ticket with INFRA. --- - To unsubscribe, e-mail: reviews-unsubscr...@spark.apache.org For additional commands, e-mail: reviews-h...@spark.apache.org