Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
Show all changes
47 commits
Select commit Hold shift + click to select a range
4a6f903
Reuse completeNextStageWithFetchFailure
beliefer Jun 19, 2020
96456e2
Merge remote-tracking branch 'upstream/master'
beliefer Jul 1, 2020
4314005
Merge remote-tracking branch 'upstream/master'
beliefer Jul 3, 2020
d6af4a7
Merge remote-tracking branch 'upstream/master'
beliefer Jul 9, 2020
f69094f
Merge remote-tracking branch 'upstream/master'
beliefer Jul 16, 2020
b86a42d
Merge remote-tracking branch 'upstream/master'
beliefer Jul 25, 2020
2ac5159
Merge branch 'master' of github.com:beliefer/spark
beliefer Jul 25, 2020
9021d6c
Merge remote-tracking branch 'upstream/master'
beliefer Jul 28, 2020
74a2ef4
Merge branch 'master' of github.com:beliefer/spark
beliefer Jul 28, 2020
9828158
Merge remote-tracking branch 'upstream/master'
beliefer Jul 31, 2020
9cd1aaf
Merge remote-tracking branch 'upstream/master'
beliefer Aug 5, 2020
abfcbb9
Merge remote-tracking branch 'upstream/master'
beliefer Aug 26, 2020
07c6c81
Merge remote-tracking branch 'upstream/master'
beliefer Sep 1, 2020
580130b
Merge remote-tracking branch 'upstream/master'
beliefer Sep 2, 2020
3712808
Merge branch 'master' of github.com:beliefer/spark
beliefer Sep 11, 2020
6107413
Merge remote-tracking branch 'upstream/master'
beliefer Sep 11, 2020
4b799b4
Merge remote-tracking branch 'upstream/master'
beliefer Sep 14, 2020
ee0ecbf
Merge remote-tracking branch 'upstream/master'
beliefer Sep 18, 2020
596bc61
Merge remote-tracking branch 'upstream/master'
beliefer Sep 24, 2020
0164e2f
Merge remote-tracking branch 'upstream/master'
beliefer Sep 27, 2020
90b79fc
Merge remote-tracking branch 'upstream/master'
beliefer Sep 29, 2020
2cef3a9
Merge remote-tracking branch 'upstream/master'
beliefer Oct 13, 2020
c26b64f
Merge remote-tracking branch 'upstream/master'
beliefer Oct 19, 2020
2e02cd2
Merge remote-tracking branch 'upstream/master'
beliefer Oct 22, 2020
a6d0741
Merge remote-tracking branch 'upstream/master'
beliefer Oct 28, 2020
82e5b2c
Merge remote-tracking branch 'upstream/master'
beliefer Nov 4, 2020
70bbf5d
Merge remote-tracking branch 'upstream/master'
beliefer Nov 6, 2020
126a51e
Merge remote-tracking branch 'upstream/master'
beliefer Nov 13, 2020
f2ceacd
Merge remote-tracking branch 'upstream/master'
beliefer Nov 19, 2020
5ad208f
Merge remote-tracking branch 'upstream/master'
beliefer Nov 23, 2020
970917e
Merge remote-tracking branch 'upstream/master'
beliefer Dec 1, 2020
086bba3
Supplement error
beliefer Dec 2, 2020
5c80da1
Update error
beliefer Dec 2, 2020
ddc1b8b
Merge remote-tracking branch 'upstream/master'
beliefer Dec 3, 2020
8585306
Update some name
beliefer Dec 4, 2020
2249e05
Adjust name
beliefer Dec 4, 2020
f198e7f
Merge branch 'master' into SPARK-32670-followup
beliefer Dec 4, 2020
2b1ed0b
Merge remote-tracking branch 'upstream/master'
beliefer Dec 4, 2020
6e8cb6e
Fix conflict
beliefer Dec 4, 2020
96325eb
Fix conflict
beliefer Dec 4, 2020
9fdfffb
Fix conflict
beliefer Dec 4, 2020
f62f297
Merge branch 'master' into SPARK-32670-followup
beliefer Dec 7, 2020
21812d0
Optimize code
beliefer Dec 7, 2020
4b4780a
Fix conflict
beliefer Dec 7, 2020
88c7eb4
Fix conflict
beliefer Dec 7, 2020
d151497
Fix break change
beliefer Dec 8, 2020
7440dad
Remove s
beliefer Dec 9, 2020
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -18,9 +18,13 @@
package org.apache.spark.sql.errors

import org.apache.spark.sql.AnalysisException
import org.apache.spark.sql.catalyst.expressions.{Expression, GroupingID}
import org.apache.spark.sql.catalyst.{FunctionIdentifier, TableIdentifier}
import org.apache.spark.sql.catalyst.analysis.ResolvedView
import org.apache.spark.sql.catalyst.expressions.{Alias, Expression, GroupingID, NamedExpression, SpecifiedWindowFrame, WindowFrame, WindowFunction, WindowSpecDefinition}
import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
import org.apache.spark.sql.catalyst.trees.TreeNode
import org.apache.spark.sql.catalyst.util.toPrettySQL
import org.apache.spark.sql.connector.catalog.CatalogV2Implicits._
import org.apache.spark.sql.connector.catalog.TableChange
import org.apache.spark.sql.internal.SQLConf
import org.apache.spark.sql.types.{AbstractDataType, DataType, StructType}
Expand All @@ -31,6 +35,7 @@ import org.apache.spark.sql.types.{AbstractDataType, DataType, StructType}
* org.apache.spark.sql.catalyst.analysis.Analyzer.
*/
object QueryCompilationErrors {

def groupingIDMismatchError(groupingID: GroupingID, groupByExprs: Seq[Expression]): Throwable = {
new AnalysisException(
s"Columns of grouping_id (${groupingID.groupByExprs.mkString(",")}) " +
Expand Down Expand Up @@ -159,6 +164,166 @@ object QueryCompilationErrors {
s"Couldn't find the reference column for $after at $parentName")
}

}
def windowSpecificationNotDefinedError(windowName: String): Throwable = {
new AnalysisException(s"Window specification $windowName is not defined in the WINDOW clause.")
}

def selectExprNotInGroupByError(expr: Expression, groupByAliases: Seq[Alias]): Throwable = {
new AnalysisException(s"$expr doesn't show up in the GROUP BY list $groupByAliases")
}

def groupingMustWithGroupingSetsOrCubeOrRollupError(): Throwable = {
new AnalysisException("grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup")
}

def pandasUDFAggregateNotSupportedInPivotError(): Throwable = {
new AnalysisException("Pandas UDF aggregate expressions are currently not supported in pivot.")
}

def aggregateExpressionRequiredForPivotError(sql: String): Throwable = {
new AnalysisException(s"Aggregate expression required for pivot, but '$sql' " +
"did not appear in any aggregate function.")
}

def expectTableNotTempViewError(quoted: String, cmd: String, t: TreeNode[_]): Throwable = {
new AnalysisException(s"$quoted is a temp view. '$cmd' expects a table",
t.origin.line, t.origin.startPosition)
}

def expectTableOrPermanentViewNotTempViewError(
quoted: String, cmd: String, t: TreeNode[_]): Throwable = {
new AnalysisException(s"$quoted is a temp view. '$cmd' expects a table or permanent view.",
t.origin.line, t.origin.startPosition)
}

def viewDepthExceedsMaxResolutionDepthError(
identifier: TableIdentifier, maxNestedViewDepth: Int, t: TreeNode[_]): Throwable = {
new AnalysisException(s"The depth of view $identifier exceeds the maximum " +
s"view resolution depth ($maxNestedViewDepth). Analysis is aborted to " +
s"avoid errors. Increase the value of ${SQLConf.MAX_NESTED_VIEW_DEPTH.key} to work " +
"around this.", t.origin.line, t.origin.startPosition)
}

def insertIntoViewNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
new AnalysisException(s"Inserting into a view is not allowed. View: $identifier.",
t.origin.line, t.origin.startPosition)
}

def writeIntoViewNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
new AnalysisException(s"Writing into a view is not allowed. View: $identifier.",
t.origin.line, t.origin.startPosition)
}

def writeIntoV1TableNotAllowedError(identifier: TableIdentifier, t: TreeNode[_]): Throwable = {
new AnalysisException(s"Cannot write into v1 table: $identifier.",
t.origin.line, t.origin.startPosition)
}

def expectTableNotViewError(v: ResolvedView, cmd: String, t: TreeNode[_]): Throwable = {
val viewStr = if (v.isTemp) "temp view" else "view"
new AnalysisException(s"${v.identifier.quoted} is a $viewStr. '$cmd' expects a table.",
t.origin.line, t.origin.startPosition)
}

def starNotAllowedWhenGroupByOrdinalPositionUsedError(): Throwable = {
new AnalysisException(
"Star (*) is not allowed in select list when GROUP BY ordinal position is used")
}

def invalidStarUsageError(prettyName: String): Throwable = {
new AnalysisException(s"Invalid usage of '*' in $prettyName")
}

def orderByPositionRangeError(index: Int, size: Int, t: TreeNode[_]): Throwable = {
new AnalysisException(s"ORDER BY position $index is not in select list " +
s"(valid range is [1, $size])", t.origin.line, t.origin.startPosition)
}

def groupByPositionRangeError(index: Int, size: Int, t: TreeNode[_]): Throwable = {
new AnalysisException(s"GROUP BY position $index is not in select list " +
s"(valid range is [1, $size])", t.origin.line, t.origin.startPosition)
}

def generatorNotExpectedError(name: FunctionIdentifier, classCanonicalName: String): Throwable = {
new AnalysisException(s"$name is expected to be a generator. However, " +
s"its class is $classCanonicalName, which is not a generator.")
}

def distinctOrFilterOnlyWithAggregateFunctionError(prettyName: String): Throwable = {
new AnalysisException("DISTINCT or FILTER specified, " +
s"but $prettyName is not an aggregate function")
}

def nonDeterministicFilterInAggregateError(): Throwable = {
new AnalysisException("FILTER expression is non-deterministic, " +
"it cannot be used in aggregate functions")
}

def aliasNumberNotMatchColumnNumberError(
columnSize: Int, outputSize: Int, t: TreeNode[_]): Throwable = {
new AnalysisException("Number of column aliases does not match number of columns. " +
s"Number of column aliases: $columnSize; " +
s"number of columns: $outputSize.", t.origin.line, t.origin.startPosition)
}

def aliasesNumberNotMatchUDTFOutputError(
aliasesSize: Int, aliasesNames: String): Throwable = {
new AnalysisException("The number of aliases supplied in the AS clause does not " +
s"match the number of columns output by the UDTF expected $aliasesSize " +
s"aliases but got $aliasesNames ")
}

def windowAggregateFunctionWithFilterNotSupportedError(): Throwable = {
new AnalysisException("window aggregate function with filter predicate is not supported yet.")
}

def windowFunctionInsideAggregateFunctionNotAllowedError(): Throwable = {
new AnalysisException("It is not allowed to use a window function inside an aggregate " +
"function. Please use the inner window function in a sub-query.")
}

def expressionWithoutWindowExpressionError(expr: NamedExpression): Throwable = {
new AnalysisException(s"$expr does not have any WindowExpression.")
}

def expressionWithMultiWindowExpressionsError(
expr: NamedExpression, distinctWindowSpec: Seq[WindowSpecDefinition]): Throwable = {
new AnalysisException(s"$expr has multiple Window Specifications ($distinctWindowSpec)." +
"Please file a bug report with this error message, stack trace, and the query.")
}

def windowFunctionNotAllowedError(clauseName: String): Throwable = {
new AnalysisException(s"It is not allowed to use window functions inside $clauseName clause")
}

def cannotSpecifyWindowFrameError(prettyName: String): Throwable = {
new AnalysisException(s"Cannot specify window frame for $prettyName function")
}

def windowFrameNotMatchRequiredFrameError(
f: SpecifiedWindowFrame, required: WindowFrame): Throwable = {
new AnalysisException(s"Window Frame $f must match the required frame $required")
}

def windowFunctionWithWindowFrameNotOrderedError(wf: WindowFunction): Throwable = {
new AnalysisException(s"Window function $wf requires window to be ordered, please add " +
s"ORDER BY clause. For example SELECT $wf(value_expr) OVER (PARTITION BY window_partition " +
"ORDER BY window_ordering) from table")
}

def cannotResolveUserSpecifiedColumnsError(col: String, t: TreeNode[_]): Throwable = {
new AnalysisException(s"Cannot resolve column name $col", t.origin.line, t.origin.startPosition)
}

def writeTableWithMismatchedColumnsError(
columnSize: Int, outputSize: Int, t: TreeNode[_]): Throwable = {
new AnalysisException("Cannot write to table due to mismatched user specified column " +
s"size($columnSize) and data column size($outputSize)", t.origin.line, t.origin.startPosition)
}

def multiTimeWindowExpressionsNotSupportedError(t: TreeNode[_]): Throwable = {
new AnalysisException("Multiple time window expressions would result in a cartesian product " +
"of rows, therefore they are currently not supported.", t.origin.line, t.origin.startPosition)
}

}
Loading