Uploaded image for project: 'Spark'
  1. Spark
  2. SPARK-20700

InferFiltersFromConstraints stackoverflows for query (v2)

    XMLWordPrintableJSON

Details

    • Bug
    • Status: Resolved
    • Major
    • Resolution: Fixed
    • 2.2.0
    • 2.2.0
    • Optimizer, SQL
    • None

    Description

      The following (complicated) query eventually fails with a stack overflow during optimization:

      CREATE TEMPORARY VIEW table_5(varchar0002_col_1, smallint_col_2, float_col_3, int_col_4, string_col_5, timestamp_col_6, string_col_7) AS VALUES
        ('68', CAST(NULL AS SMALLINT), CAST(244.90413 AS FLOAT), -137, '571', TIMESTAMP('2015-01-14 00:00:00.0'), '947'),
        ('82', CAST(213 AS SMALLINT), CAST(53.184647 AS FLOAT), -724, '-278', TIMESTAMP('1999-08-15 00:00:00.0'), '437'),
        ('-7', CAST(-15 AS SMALLINT), CAST(NULL AS FLOAT), -890, '778', TIMESTAMP('1991-05-23 00:00:00.0'), '630'),
        ('22', CAST(676 AS SMALLINT), CAST(385.27386 AS FLOAT), CAST(NULL AS INT), '-10', TIMESTAMP('1996-09-29 00:00:00.0'), '641'),
        ('16', CAST(430 AS SMALLINT), CAST(187.23717 AS FLOAT), 989, CAST(NULL AS STRING), TIMESTAMP('2024-04-21 00:00:00.0'), '-234'),
        ('83', CAST(760 AS SMALLINT), CAST(-695.45386 AS FLOAT), -970, '330', CAST(NULL AS TIMESTAMP), '-740'),
        ('68', CAST(-930 AS SMALLINT), CAST(NULL AS FLOAT), -915, '-766', CAST(NULL AS TIMESTAMP), CAST(NULL AS STRING)),
        ('48', CAST(692 AS SMALLINT), CAST(-220.59615 AS FLOAT), 940, '-514', CAST(NULL AS TIMESTAMP), '181'),
        ('21', CAST(44 AS SMALLINT), CAST(NULL AS FLOAT), -175, '761', TIMESTAMP('2016-06-30 00:00:00.0'), '487'),
        ('50', CAST(953 AS SMALLINT), CAST(837.2948 AS FLOAT), 705, CAST(NULL AS STRING), CAST(NULL AS TIMESTAMP), '-62');
      
      CREATE VIEW bools(a, b) as values (1, true), (1, true), (1, null);
      
      SELECT
      AVG(-13) OVER (ORDER BY COUNT(t1.smallint_col_2) DESC ROWS 27 PRECEDING ) AS float_col,
      COUNT(t1.smallint_col_2) AS int_col
      FROM table_5 t1
      INNER JOIN (
      SELECT
      (MIN(-83) OVER (PARTITION BY t2.a ORDER BY t2.a, (t1.int_col_4) * (t1.int_col_4) ROWS BETWEEN CURRENT ROW AND 15 FOLLOWING)) NOT IN (-222, 928) AS boolean_col,
      t2.a,
      (t1.int_col_4) * (t1.int_col_4) AS int_col
      FROM table_5 t1
      LEFT JOIN bools t2 ON (t2.a) = (t1.int_col_4)
      WHERE
      (t1.smallint_col_2) > (t1.smallint_col_2)
      GROUP BY
      t2.a,
      (t1.int_col_4) * (t1.int_col_4)
      HAVING
      ((t1.int_col_4) * (t1.int_col_4)) IN ((t1.int_col_4) * (t1.int_col_4), SUM(t1.int_col_4))
      ) t2 ON (((t2.int_col) = (t1.int_col_4)) AND ((t2.a) = (t1.int_col_4))) AND ((t2.a) = (t1.smallint_col_2));
      

      (I haven't tried to minimize this failing case yet).

      Based on sampled jstacks from the driver, it looks like the query might be repeatedly inferring filters from constraints and then pruning those filters.

      Here's part of the stack at the point where it stackoverflows:

      [... repeats ...]
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
              at scala.collection.immutable.List.flatMap(List.scala:344)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
              at scala.collection.immutable.List.flatMap(List.scala:344)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
              at scala.collection.immutable.List.flatMap(List.scala:344)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
              at scala.collection.immutable.List.flatMap(List.scala:344)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
              at scala.collection.immutable.List.flatMap(List.scala:344)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$$anonfun$org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative$1.apply(Canonicalize.scala:50)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.TraversableLike$$anonfun$flatMap$1.apply(TraversableLike.scala:241)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.flatMap(TraversableLike.scala:241)
              at scala.collection.immutable.List.flatMap(List.scala:344)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.org$apache$spark$sql$catalyst$expressions$Canonicalize$$gatherCommutative(Canonicalize.scala:50)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.orderCommutative(Canonicalize.scala:58)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.expressionReorder(Canonicalize.scala:63)
              at org.apache.spark.sql.catalyst.expressions.Canonicalize$.execute(Canonicalize.scala:36)
              at org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:158)
              - locked <0x00000007a298b940> (a org.apache.spark.sql.catalyst.expressions.Multiply)
              at org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:156)
              at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:157)
              at org.apache.spark.sql.catalyst.expressions.Expression$$anonfun$1.apply(Expression.scala:157)
              at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
      [...]
       at scala.collection.TraversableLike$$anonfun$map$1.apply(TraversableLike.scala:234)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at scala.collection.TraversableLike$class.map(TraversableLike.scala:234)
              at scala.collection.immutable.List.map(List.scala:285)
              at org.apache.spark.sql.catalyst.expressions.Expression.canonicalized$lzycompute(Expression.scala:157)
              - locked <0x00000007a28b7170> (a org.apache.spark.sql.catalyst.expressions.EqualNullSafe)
              at org.apache.spark.sql.catalyst.expressions.Expression.canonicalized(Expression.scala:156)
              at org.apache.spark.sql.catalyst.expressions.ExpressionSet.add(ExpressionSet.scala:56)
              at org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus(ExpressionSet.scala:66)
              at org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus(ExpressionSet.scala:50)
              at scala.collection.SetLike$$anonfun$$plus$plus$1.apply(SetLike.scala:141)
              at scala.collection.SetLike$$anonfun$$plus$plus$1.apply(SetLike.scala:141)
              at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
              at scala.collection.TraversableOnce$$anonfun$foldLeft$1.apply(TraversableOnce.scala:157)
              at scala.collection.Iterator$class.foreach(Iterator.scala:893)
              at scala.collection.AbstractIterator.foreach(Iterator.scala:1336)
              at scala.collection.IterableLike$class.foreach(IterableLike.scala:72)
              at scala.collection.AbstractIterable.foreach(Iterable.scala:54)
              at scala.collection.immutable.HashSet$HashSetCollision1.foreach(HashSet.scala:462)
              at scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:972)
              at scala.collection.immutable.HashSet$HashTrieSet.foreach(HashSet.scala:972)
              at scala.collection.TraversableOnce$class.foldLeft(TraversableOnce.scala:157)
              at scala.collection.AbstractTraversable.foldLeft(Traversable.scala:104)
              at scala.collection.TraversableOnce$class.$div$colon(TraversableOnce.scala:151)
              at scala.collection.AbstractTraversable.$div$colon(Traversable.scala:104)
              at scala.collection.SetLike$class.$plus$plus(SetLike.scala:141)
              at org.apache.spark.sql.catalyst.expressions.ExpressionSet.$plus$plus(ExpressionSet.scala:50)
              at scala.collection.SetLike$class.union(SetLike.scala:163)
              at org.apache.spark.sql.catalyst.expressions.ExpressionSet.union(ExpressionSet.scala:50)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.getRelevantConstraints(QueryPlan.scala:35)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
              - locked <0x000000079a7ebaa8> (a org.apache.spark.sql.catalyst.plans.logical.Join)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
              at org.apache.spark.sql.catalyst.plans.logical.Filter.validConstraints(basicLogicalOperators.scala:138)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
              - locked <0x000000079a7eba58> (a org.apache.spark.sql.catalyst.plans.logical.Filter)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
              at org.apache.spark.sql.catalyst.plans.logical.Aggregate.validConstraints(basicLogicalOperators.scala:571)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
              - locked <0x000000079a7eb958> (a org.apache.spark.sql.catalyst.plans.logical.Aggregate)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
              at org.apache.spark.sql.catalyst.plans.logical.Filter.validConstraints(basicLogicalOperators.scala:138)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
              - locked <0x000000079a7eb908> (a org.apache.spark.sql.catalyst.plans.logical.Filter)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
              at org.apache.spark.sql.catalyst.plans.logical.Project.validConstraints(basicLogicalOperators.scala:65)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
              - locked <0x000000079a7eb328> (a org.apache.spark.sql.catalyst.plans.logical.Project)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
              at org.apache.spark.sql.catalyst.plans.logical.Join.validConstraints(basicLogicalOperators.scala:320)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints$lzycompute(QueryPlan.scala:187)
              - locked <0x000000079a7eb2c0> (a org.apache.spark.sql.catalyst.plans.logical.Join)
              at org.apache.spark.sql.catalyst.plans.QueryPlan.constraints(QueryPlan.scala:187)
              at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$inferFilters$1.applyOrElse(Optimizer.scala:642)
              at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints$$anonfun$inferFilters$1.applyOrElse(Optimizer.scala:629)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$2.apply(TreeNode.scala:267)
              at org.apache.spark.sql.catalyst.trees.CurrentOrigin$.withOrigin(TreeNode.scala:70)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:266)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$transformDown$1.apply(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode$$anonfun$4.apply(TreeNode.scala:306)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapProductIterator(TreeNode.scala:187)
              at org.apache.spark.sql.catalyst.trees.TreeNode.mapChildren(TreeNode.scala:304)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transformDown(TreeNode.scala:272)
              at org.apache.spark.sql.catalyst.trees.TreeNode.transform(TreeNode.scala:256)
              at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints.inferFilters(Optimizer.scala:629)
              at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints.apply(Optimizer.scala:623)
              at org.apache.spark.sql.catalyst.optimizer.InferFiltersFromConstraints.apply(Optimizer.scala:620)
              at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:85)
              at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1$$anonfun$apply$1.apply(RuleExecutor.scala:82)
              at scala.collection.LinearSeqOptimized$class.foldLeft(LinearSeqOptimized.scala:124)
              at scala.collection.immutable.List.foldLeft(List.scala:84)
              at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:82)
              at org.apache.spark.sql.catalyst.rules.RuleExecutor$$anonfun$execute$1.apply(RuleExecutor.scala:74)
              at scala.collection.immutable.List.foreach(List.scala:381)
              at org.apache.spark.sql.catalyst.rules.RuleExecutor.execute(RuleExecutor.scala:74)
              at org.apache.spark.sql.execution.QueryExecution.optimizedPlan$lzycompute(QueryExecution.scala:79)
              - locked <0x0000000787ea2848> (a org.apache.spark.sql.execution.QueryExecution)
              at org.apache.spark.sql.execution.QueryExecution.optimizedPlan(QueryExecution.scala:79)
              at org.apache.spark.sql.execution.QueryExecution.sparkPlan$lzycompute(QueryExecution.scala:85)
              - locked <0x0000000787ea2848> (a org.apache.spark.sql.execution.QueryExecution)
              at org.apache.spark.sql.execution.QueryExecution.sparkPlan(QueryExecution.scala:81)
              at org.apache.spark.sql.execution.QueryExecution.executedPlan$lzycompute(QueryExecution.scala:90)
              - locked <0x0000000787ea2848> (a org.apache.spark.sql.execution.QueryExecution)
      [...]
      

      I suspect this is similar to SPARK-17733, another bug where InferFiltersFromConstraints, so I'll cc jiangxb1987 and sameerag who worked on that earlier fix.

      Attachments

        Issue Links

          Activity

            People

              jiangxb1987 Xingbo Jiang
              joshrosen Josh Rosen
              Votes:
              0 Vote for this issue
              Watchers:
              3 Start watching this issue

              Dates

                Created:
                Updated:
                Resolved: