Error Conditions
This is a list of error states and conditions that may be returned by Spark SQL.
Error State / SQLSTATE | Error Condition & Sub-Condition | Message |
---|---|---|
07001 |
#
ALL |
Using name parameterized queries requires all parameters to be named. Parameters missing names: |
07501 |
#
INVALID |
The INTO clause of EXECUTE IMMEDIATE is only valid for queries but the given statement is not a query: |
07501 |
#
NESTED |
Nested EXECUTE IMMEDIATE commands are not allowed. Please ensure that the SQL query provided ( |
0A000 |
#
CANNOT |
Dataset transformations and actions can only be invoked by the driver, not inside of other Dataset transformations; for example, dataset1.map(x => dataset2.values.count() * x) is invalid because the values transformation and count action cannot be performed inside of the dataset1.map transformation. For more information, see SPARK-28702. |
0A000 |
#
CANNOT |
Cannot save variant data type into external storage. |
0A000 |
#
CANNOT |
Cannot update |
#
ARRAY |
Update the element by updating |
|
#
INTERVAL |
Update an interval by updating its fields. |
|
#
MAP |
Update a map by updating |
|
#
STRUCT |
Update a struct by updating its fields. |
|
#
USER |
Update a UserDefinedType[ |
|
0A000 |
#
CLASS |
|
0A000 |
#
CONCURRENT |
Another instance of this query was just started by a concurrent session. |
0A000 |
#
CREATE |
Not allowed to create the permanent view |
0A000 |
#
DISTINCT |
Distinct window functions are not supported: |
0A000 |
#
INVALID |
The group aggregate pandas UDF |
0A000 |
#
INVALID |
Cannot use |
0A000 |
#
MULTI |
Not allowed to implement multiple UDF interfaces, UDF class |
0A000 |
#
NAMED |
Cannot call function |
0A000 |
#
NOT |
ALTER TABLE ALTER/CHANGE COLUMN is not supported for changing |
0A000 |
#
NOT |
|
0A000 |
#
NOT |
|
0A000 |
#
NOT |
Not supported command in JDBC catalog: |
#
COMMAND
|
|
|
#
COMMAND |
|
|
0A000 |
#
SCALAR |
The correlated scalar subquery ' |
0A000 |
#
STAR |
Star (*) is not allowed in a select list when GROUP BY an ordinal position is used. |
0A000 |
#
UNSUPPORTED |
Don't support add file. |
#
DIRECTORY
|
The file |
|
#
LOCAL |
The local directory |
|
0A000 |
#
UNSUPPORTED |
Unsupported arrow type |
0A000 |
#
UNSUPPORTED |
Cannot call the method " |
#
FIELD |
The row shall have a schema to get an index of the field |
|
#
WITHOUT |
||
0A000 |
#
UNSUPPORTED |
The char/varchar type can't be used in the table schema. If you want Spark treat them as string type as same as Spark 3.0 and earlier, please set "spark.sql.legacy.charVarcharAsString" to "true". |
0A000 |
#
UNSUPPORTED |
Collation |
#
FOR |
function |
|
0A000 |
#
UNSUPPORTED |
Unsupported data source type for direct query on files: |
0A000 |
#
UNSUPPORTED |
Unsupported data type |
0A000 |
#
UNSUPPORTED |
The data source " |
0A000 |
#
UNSUPPORTED |
The |
0A000 |
#
UNSUPPORTED |
Cannot create encoder for |
0A000 |
#
UNSUPPORTED |
DEFAULT column values is not supported. |
#
WITHOUT |
||
#
WITH |
Enable it by setting "spark.sql.defaultColumn.enabled" to "true". |
|
0A000 |
#
UNSUPPORTED |
The deserializer is not supported: |
#
DATA |
need a(n) |
|
#
FIELD |
try to map |
|
0A000 |
#
UNSUPPORTED |
The feature is not supported: |
#
AES |
AES- |
|
#
AES |
|
|
#
AES |
|
|
#
ANALYZE |
The ANALYZE TABLE FOR COLUMNS command can operate on temporary views that have been cached already. Consider to cache the view |
|
#
ANALYZE |
The ANALYZE TABLE FOR COLUMNS command does not support the type |
|
#
ANALYZE |
The ANALYZE TABLE command does not support views. |
|
#
CATALOG |
Catalog |
|
#
COLLATION
|
Collation is not yet supported. |
|
#
COMBINATION |
Combination of ORDER BY/SORT BY/DISTRIBUTE BY/CLUSTER BY. |
|
#
COMMENT |
Attach a comment to the namespace |
|
#
DESC |
DESC TABLE COLUMN for a specific partition. |
|
#
DROP |
Drop the default database |
|
#
DROP |
Drop the namespace |
|
#
HIVE |
The |
|
#
HIVE |
Hive table |
|
#
INSERT |
INSERT INTO |
|
#
LATERAL |
Referencing a lateral column alias |
|
#
LATERAL |
Referencing lateral column alias |
|
#
LATERAL |
Referencing a lateral column alias via GROUP BY alias/ALL is not supported yet. |
|
#
LATERAL |
Referencing a lateral column alias |
|
#
LATERAL |
JOIN USING with LATERAL correlation. |
|
#
LITERAL |
Literal for ' |
|
#
MULTIPLE |
Multiple bucket TRANSFORMs. |
|
#
MULTI |
The target JDBC server hosting table |
|
#
ORC |
Unable to convert |
|
#
OVERWRITE |
INSERT OVERWRITE with a subquery condition. |
|
#
PANDAS |
Pandas user defined aggregate function in the PIVOT clause. |
|
#
PARAMETER |
Parameter markers are not allowed in |
|
#
PARTITION |
Invalid partitioning: |
|
#
PIVOT |
PIVOT clause following a GROUP BY clause. Consider pushing the GROUP BY into a subquery. |
|
#
PIVOT |
Pivoting by the value ' |
|
#
PURGE |
Partition purge. |
|
#
PURGE |
Purge table. |
|
#
PYTHON |
Python UDF in the ON clause of a |
|
#
REMOVE |
Remove a comment from the namespace |
|
#
REPLACE |
The replace function does not support nested column |
|
#
SET |
|
|
#
SET |
Cannot have MAP type columns in DataFrame which calls set operations (INTERSECT, EXCEPT, etc.), but the type of column |
|
#
SET |
set PROPERTIES and DBPROPERTIES at the same time. |
|
#
SET |
|
|
#
SET |
|
|
#
STATE |
Creating multiple column families with |
|
#
STATE |
Removing column families with |
|
#
STATE |
State TTL with |
|
#
TABLE |
Table |
|
#
TEMPORARY |
Temporary views cannot be created with the WITH SCHEMA clause. Recreate the temporary view when the underlying schema changes, or use a persisted view. |
|
#
TIME |
Time travel on the relation: |
|
#
TOO |
UDF class with |
|
#
TRANSFORM |
TRANSFORM with the DISTINCT/ALL clause. |
|
#
TRANSFORM |
TRANSFORM with SERDE is only supported in hive mode. |
|
0A000 |
#
UNSUPPORTED |
The save mode |
#
EXISTENT |
an existent path. |
|
#
NON |
a non-existent path. |
|
0A000 |
#
UNSUPPORTED |
Unsupported subquery expression: |
#
ACCESSING |
Accessing outer query column is not allowed in this location: |
|
#
AGGREGATE |
Found an aggregate function in a correlated predicate that has both outer and local references, which is not supported: |
|
#
CORRELATED |
Correlated column is not allowed in predicate: |
|
#
CORRELATED |
A correlated outer name reference within a subquery expression body was not found in the enclosing query: |
|
#
CORRELATED |
Expressions referencing the outer query are not supported outside of WHERE/HAVING clauses: |
|
#
HIGHER |
Subquery expressions are not supported within higher-order functions. Please remove all subquery expressions from higher-order functions and then try the query again. |
|
#
LATERAL |
Lateral join condition cannot be non-deterministic: |
|
#
MUST |
Correlated scalar subqueries must be aggregated to return at most one row. |
|
#
NON |
A GROUP BY clause in a scalar correlated subquery cannot contain non-correlated columns: |
|
#
NON |
Non-deterministic lateral subqueries are not supported when joining with outer relations that produce more than one row: |
|
#
UNSUPPORTED |
Correlated subqueries in the join predicate cannot reference both join inputs: |
|
#
UNSUPPORTED |
Correlated column reference ' |
|
#
UNSUPPORTED |
Correlated scalar subqueries can only be used in filters, aggregations, projections, and UPDATE/MERGE/DELETE commands: |
|
#
UNSUPPORTED |
IN/EXISTS predicate subqueries can only be used in filters, joins, aggregations, window functions, projections, and UPDATE/MERGE/DELETE commands: |
|
#
UNSUPPORTED |
Table arguments are used in a function where they are not supported: |
|
0A000 |
#
UNSUPPORTED |
Literals of the type |
0AKD0 |
#
CANNOT |
Renaming a |
21000 |
#
ROW |
More than one row returned by a subquery used as a row. |
21000 |
#
SCALAR |
More than one row returned by a subquery used as an expression. |
21S01 |
#
CREATE |
Cannot create view |
#
NOT |
not enough data columns: View columns: |
|
#
TOO |
too many data columns: View columns: |
|
21S01 |
#
INSERT |
Cannot write to |
#
NOT |
not enough data columns: Table columns: |
|
#
TOO |
too many data columns: Table columns: |
|
21S01 |
#
INSERT |
Cannot write to ' |
22000 |
#
HLL |
Sketches have different |
22003 |
#
ARITHMETIC |
|
22003 |
#
BINARY |
|
22003 |
#
CAST |
The value |
22003 |
#
CAST |
Fail to assign a value of |
22003 |
#
DECIMAL |
Decimal precision |
22003 |
#
INCORRECT |
Max offset with |
22003 |
#
INVALID |
The index |
22003 |
#
INVALID |
The index |
22003 |
#
INVALID |
The 0-indexed bitmap position |
22003 |
#
INVALID |
The boundary |
#
END
|
Expected the value is '0', ' |
|
#
START
|
Expected the value is '0', ' |
|
22003 |
#
INVALID |
The index 0 is invalid. An index shall be either |
22003 |
#
INVALID |
Numeric literal |
22003 |
#
NUMERIC |
The value |
22003 |
#
NUMERIC |
|
#
WITHOUT |
The |
|
#
WITH |
|
|
22003 |
#
SUM |
The sum of the LIMIT clause and the OFFSET clause must not be greater than the maximum 32-bit integer value (2,147,483,647) but found limit = |
22004 |
#
COMPARATOR |
The comparator has returned a NULL for a comparison between |
22004 |
#
NULL |
Execute immediate requires a non-null variable as the query string, but the provided variable |
22006 |
#
CANNOT |
Unable to parse |
22006 |
#
INVALID |
Error parsing ' |
#
ARITHMETIC |
Uncaught arithmetic exception while parsing ' |
|
#
INPUT |
Interval string cannot be empty. |
|
#
INPUT |
Interval string cannot be null. |
|
#
INVALID |
|
|
#
INVALID |
Interval can only support nanosecond precision, |
|
#
INVALID |
Invalid interval prefix |
|
#
INVALID |
Invalid unit |
|
#
INVALID |
Invalid value |
|
#
MISSING |
Expect a number after |
|
#
MISSING |
Expect a unit name after |
|
#
UNKNOWN |
Unknown error when parsing |
|
#
UNRECOGNIZED |
Unrecognized number |
|
22007 |
#
CANNOT |
|
22007 |
#
INVALID |
Unrecognized datetime pattern: |
#
ILLEGAL |
Illegal pattern character found in datetime pattern: |
|
#
LENGTH
|
Too many letters in datetime pattern: |
|
22008 |
#
DATETIME |
Datetime operation overflow: |
2200E |
#
NULL |
Cannot use null as map key. |
22012 |
#
DIVIDE |
Division by zero. Use |
22012 |
#
INTERVAL |
Division by zero. Use |
22015 |
#
INTERVAL |
|
22018 |
#
CANNOT |
Cannot parse decimal. Please ensure that the input is a valid number with optional decimal point or comma separators. |
22018 |
#
CANNOT |
Error parsing descriptor bytes into Protobuf FileDescriptorSet. |
22018 |
#
CAST |
The value |
22018 |
#
CONVERSION |
The value |
22018 |
#
FAILED |
Failed parsing struct: |
2201E |
#
STRUCT |
Input row doesn't have expected number of values required by the schema. |
22022 |
#
INVALID |
The value ' |
#
DEFAULT |
Cannot resolve the given default collation. Did you mean ' |
|
#
TIME |
Cannot resolve the given timezone. |
|
22023 |
#
INVALID |
The fraction of sec must be zero. Valid range is [0, 60]. If necessary set |
22023 |
#
INVALID |
The value of parameter(s) |
#
AES |
detail message: |
|
#
AES |
supports 16-byte CBC IVs and 12-byte GCM IVs, but got |
|
#
AES |
expects a binary value with 16, 24 or 32 bytes, but got |
|
#
BINARY |
expects one of binary formats 'base64', 'hex', 'utf-8', but got |
|
#
BIT |
expects an integer value in [0, |
|
#
CHARSET
|
expects one of the charsets 'US-ASCII', 'ISO-8859-1', 'UTF-8', 'UTF-16BE', 'UTF-16LE', 'UTF-16', but got |
|
#
DATETIME |
expects one of the units without quotes YEAR, QUARTER, MONTH, WEEK, DAY, DAYOFYEAR, HOUR, MINUTE, SECOND, MILLISECOND, MICROSECOND, but got the string literal |
|
#
LENGTH
|
Expects |
|
#
NULL
|
expects a non-NULL value. |
|
#
PATTERN
|
|
|
#
REGEX |
Expects group index between 0 and |
|
#
START
|
Expects a positive or a negative value for |
|
#
ZERO |
expects %1$, %2$ and so on, but got %0$. |
|
22023 |
#
INVALID |
The variant value |
22023 |
#
INVALID |
The path |
22023 |
#
MALFORMED |
Malformed records are detected in record parsing: |
#
CANNOT |
Parsing JSON arrays as structs is forbidden. |
|
#
CANNOT |
Cannot parse the value |
|
#
WITHOUT |
||
22023 |
#
MALFORMED |
Variant binary is malformed. Please check the data source is valid. |
22023 |
#
RULE |
Not found an id for the rule name " |
22023 |
#
SECOND |
The second argument of |
22023 |
#
TABLE |
Failed to evaluate the table function |
22023 |
#
TABLE |
Failed to evaluate the table function |
22023 |
#
VARIANT |
Cannot construct a Variant larger than 16 MiB. The maximum allowed size of a Variant value is 16 MiB. |
22023 |
#
VARIANT |
Failed to build variant because of a duplicate object key |
22023 |
#
VARIANT |
Cannot build variant bigger than |
22032 |
#
INVALID |
Cannot convert JSON root field to target Spark type. |
22032 |
#
INVALID |
Input schema |
2203G |
#
CANNOT |
Cannot parse the field name |
2203G |
#
FAILED |
Failed to convert the row value |
2203G |
#
INVALID |
Failed to convert the JSON string ' |
2203G |
#
INVALID |
Collations can only be applied to string types, but the JSON data type is |
22546 |
#
CANNOT |
The provided URL cannot be decoded: |
22546 |
#
HLL |
Invalid call to |
22546 |
#
HLL |
Invalid call to |
22KD3 |
#
AVRO |
Cannot convert Avro |
22P02 |
#
INVALID |
The url is invalid: |
22P03 |
#
INVALID |
The expected format is ByteString, but was |
23505 |
#
DUPLICATED |
Duplicate map key |
23505 |
#
DUPLICATE |
Found duplicate keys |
23K01 |
#
MERGE |
The ON search condition of the MERGE statement matched a single row from the target table with multiple rows of the source table. This could result in the target row being operated on more than once with an update or delete operation and is not allowed. |
2BP01 |
#
SCHEMA |
Cannot drop a schema |
38000 |
#
CLASS |
|
38000 |
#
FAILED |
Failed preparing of the function |
38000 |
#
INVALID |
Function |
38000 |
#
NO |
UDF class |
38000 |
#
PYTHON |
Failed to |
38000 |
#
PYTHON |
Failed when Python streaming data source perform |
38000 |
#
TABLE |
Failed to analyze the Python user defined table function: |
39000 |
#
FAILED |
User defined function ( |
39000 |
#
FOREACH |
An error occurred in the user provided function in foreach batch sink. Reason: |
40000 |
#
CONCURRENT |
Concurrent update to the log. Multiple streaming jobs detected for |
42000 |
#
AMBIGUOUS |
Ambiguous reference to the field |
42000 |
#
INVALID |
Column or field |
42000 |
#
INVALID |
Can't extract a value from |
42000 |
#
INVALID |
Field name should be a non-null string literal, but it's |
42000 |
#
INVALID |
Field name |
42000 |
#
INVALID |
Invalid inline table. |
#
CANNOT |
Cannot evaluate the expression |
|
#
FAILED |
Failed to evaluate the SQL expression |
|
#
INCOMPATIBLE |
Found incompatible types in the column |
|
#
NUM |
Inline table expected |
|
42000 |
#
INVALID |
The specified save mode |
42000 |
#
INVALID |
Expected format is 'SET', 'SET key', or 'SET key=value'. If you want to include special characters in key, or include semicolon in value, please use backquotes, e.g., SET |
42000 |
#
INVALID |
Invalid SQL syntax: |
#
ANALYZE |
ANALYZE TABLE(S) ... COMPUTE STATISTICS ... |
|
#
CREATE |
Cannot create a routine with both IF NOT EXISTS and REPLACE specified. |
|
#
CREATE |
CREATE TEMPORARY FUNCTION with specifying a database( |
|
#
CREATE |
CREATE TEMPORARY FUNCTION with IF NOT EXISTS is not allowed. |
|
#
EMPTY |
Partition key |
|
#
FUNCTION |
The function |
|
#
INVALID |
Expected a column reference for transform |
|
#
INVALID |
Syntax error: call to table-valued function is invalid because parentheses are missing around the provided TABLE argument |
|
#
INVALID |
Table valued function cannot specify database name: |
|
#
INVALID |
Window reference |
|
#
LATERAL |
LATERAL can only be used with subquery and table-valued functions. |
|
#
MULTI |
|
|
#
OPTION |
option or property key |
|
#
REPETITIVE |
The definition of window |
|
#
SHOW |
Invalid pattern in SHOW FUNCTIONS: |
|
#
SHOW |
SHOW |
|
#
TRANSFORM |
The transform |
|
#
UNRESOLVED |
Cannot resolve window reference |
|
#
UNSUPPORTED |
Unsupported function name |
|
#
VARIABLE |
The definition of a SQL variable requires either a datatype or a DEFAULT clause. For example, use |
|
42000 |
#
INVALID |
Invalid usage of |
42000 |
#
INVALID |
The requested write distribution is invalid. |
#
PARTITION |
The partition number and advisory partition size can't be specified at the same time. |
|
#
PARTITION |
The number of partitions can't be specified with unspecified distribution. |
|
#
PARTITION |
The advisory partition size can't be specified with unspecified distribution. |
|
42000 |
#
NON |
PARTITION clause cannot contain the non-partition column: |
42000 |
#
NOT |
Assigning a NULL is not allowed here. |
#
ARRAY |
The array |
|
#
MAP |
The map |
|
42000 |
#
NO |
No handler for UDAF ' |
42000 |
#
NULLABLE |
Column or field |
42000 |
#
NULLABLE |
Row ID attributes cannot be nullable: |
42001 |
#
INVALID |
Found an invalid expression encoder. Expects an instance of ExpressionEncoder but got |
42601 |
#
COLUMN |
Column aliases are not allowed in |
42601 |
#
IDENTIFIER |
|
42601 |
#
ILLEGAL |
Illegal value provided to the State Store |
#
EMPTY |
Cannot write empty list values to State Store for StateName |
|
#
NULL |
Cannot write null values to State Store for StateName |
|
42601 |
#
INVALID |
Cannot use |
42601 |
#
INVALID |
Cannot extract |
42601 |
#
INVALID |
The format is invalid: |
#
CONT |
Thousands separators (, or G) must have digits in between them in the number format. |
|
#
CUR |
Currency characters must appear before any decimal point in the number format. |
|
#
CUR |
Currency characters must appear before digits in the number format. |
|
#
EMPTY
|
The number format string cannot be empty. |
|
#
ESC |
The escape character is not allowed to end with. |
|
#
ESC |
The escape character is not allowed to precede |
|
#
MISMATCH |
The input |
|
#
THOUSANDS |
Thousands separators (, or G) may not appear after the decimal point in the number format. |
|
#
UNEXPECTED |
Found the unexpected |
|
#
WRONG |
The format string requires at least one number digit. |
|
#
WRONG |
At most one |
|
42601 |
#
INVALID |
The partition command is invalid. |
#
PARTITION |
Table |
|
#
PARTITION |
Table |
|
42601 |
#
INVALID |
The statement or clause: |
42601 |
#
INVALID |
Cannot specify ORDER BY or a window frame for |
42601 |
#
LOCAL |
LOCAL must be used together with the schema of |
42601 |
#
NOT |
Not allowed in the FROM clause: |
#
LATERAL |
LATERAL together with PIVOT. |
|
#
LATERAL |
LATERAL together with UNPIVOT. |
|
#
UNPIVOT |
UNPIVOT together with PIVOT. |
|
42601 |
#
NOT |
The expression |
#
NOT |
To be considered constant the expression must not depend on any columns, contain a subquery, or invoke a non deterministic function such as rand(). |
|
#
NULL
|
The expression evaluates to NULL. |
|
#
WRONG |
The data type of the expression is |
|
42601 |
#
NOT |
Unresolved encoder expected, but |
42601 |
#
PARSE |
Syntax error at or near |
42601 |
#
REF |
References to DEFAULT column values are not allowed within the PARTITION clause. |
42601 |
#
SORT |
sortBy must be used together with bucketBy. |
42601 |
#
SPECIFY |
A CREATE TABLE without explicit column list cannot specify bucketing information. Please use the form with explicit column list and specify bucketing information. Alternatively, allow bucketing information to be inferred by omitting the clause. |
42601 |
#
SPECIFY |
A CREATE TABLE without explicit column list cannot specify PARTITIONED BY. Please use the form with explicit column list and specify PARTITIONED BY. Alternatively, allow partitioning to be inferred by omitting the PARTITION BY clause. |
42601 |
#
STDS |
' |
42601 |
#
SYNTAX |
Support of the clause or keyword: |
#
BANG |
The '!' keyword is only supported as an alias for the prefix operator 'NOT'. Use the 'NOT' keyword instead for infix clauses such as |
|
42601 |
#
UNCLOSED |
Found an unclosed bracketed comment. Please, append */ at the end of the comment. |
42601 |
#
WINDOW |
Window function |
42601 |
#
WRITE |
|
42602 |
#
INVALID |
Invalid value for delimiter. |
#
DELIMITER |
Delimiter cannot be more than one character: |
|
#
EMPTY |
Delimiter cannot be empty string. |
|
#
SINGLE |
Single backslash is prohibited. It has special meaning as beginning of an escape sequence. To get the backslash character, pass a string with two backslashes as the delimiter. |
|
#
UNSUPPORTED |
Unsupported special character for delimiter: |
|
42602 |
#
INVALID |
The unquoted identifier |
42602 |
#
INVALID |
|
42602 |
#
INVALID |
|
42602 |
#
INVALID |
|
42604 |
#
AS |
Invalid as-of join. |
#
TOLERANCE |
The input argument |
|
#
TOLERANCE |
The input argument |
|
42604 |
#
EMPTY |
Failed to parse an empty string for data type |
42604 |
#
INVALID |
Found an invalid escape string: |
42604 |
#
INVALID |
|
42604 |
#
INVALID |
The value of the typed literal |
42605 |
#
WRONG |
The |
#
WITHOUT |
Please, refer to ' |
|
#
WITH |
If you have to call this function with |
|
42607 |
#
NESTED |
It is not allowed to use an aggregate function in the argument of another aggregate function. Please use the inner aggregate function in a sub-query. |
42608 |
#
DEFAULT |
A DEFAULT keyword in a MERGE, INSERT, UPDATE, or SET VARIABLE command could not be directly assigned to a target column because it was part of an expression. For example: |
42608 |
#
NO |
Can't determine the default value for |
42611 |
#
CANNOT |
Watermark needs to be defined to reassign event time column. Failed to find watermark definition in the streaming query. |
42613 |
#
INCOMPATIBLE |
The join types |
42613 |
#
INVALID |
The |
42613 |
#
INVALID |
Parameterized query must either use positional, or named parameters, but not both. |
42613 |
#
INVALID |
The |
42613 |
#
NON |
When there are more than one MATCHED clauses in a MERGE statement, only the last MATCHED clause can omit the condition. |
42613 |
#
NON |
When there are more than one NOT MATCHED BY SOURCE clauses in a MERGE statement, only the last NOT MATCHED BY SOURCE clause can omit the condition. |
42613 |
#
NON |
When there are more than one NOT MATCHED [BY TARGET] clauses in a MERGE statement, only the last NOT MATCHED [BY TARGET] clause can omit the condition. |
42613 |
#
STDS |
The options |
42614 |
#
DUPLICATE |
Found duplicate clauses: |
42614 |
#
REPEATED |
The |
42616 |
#
STDS |
Invalid value for source option ' |
#
IS |
cannot be empty. |
|
#
IS |
cannot be negative. |
|
#
WITH |
|
|
42617 |
#
PARSE |
Syntax error, unexpected empty statement. |
42621 |
#
UNSUPPORTED |
Cannot create generated column |
42623 |
#
GENERATED |
A column cannot have both a default value and a generation expression but column |
42623 |
#
INVALID |
Failed to execute |
#
DATA |
which requires |
|
#
NOT |
which is not a constant expression whose equivalent value is known at query planning time. |
|
#
SUBQUERY |
which contains subquery expressions. |
|
#
UNRESOLVED |
which fails to resolve as a valid expression. |
|
42701 |
#
DUPLICATE |
The columns or variables |
42701 |
#
EXEC |
The USING clause of this EXECUTE IMMEDIATE command contained multiple arguments with same alias ( |
42702 |
#
AMBIGUOUS |
Column or field |
42702 |
#
AMBIGUOUS |
Column |
42702 |
#
AMBIGUOUS |
Lateral column alias |
42702 |
#
EXCEPT |
Columns in an EXCEPT list must be distinct and non-overlapping, but got ( |
42703 |
#
COLUMN |
|
42703 |
#
COLUMN |
The column |
42703 |
#
UNRESOLVED |
A column, variable, or function parameter with name |
#
WITHOUT |
||
#
WITH |
Did you mean one of the following? [ |
|
42703 |
#
UNRESOLVED |
A field with name |
#
WITHOUT |
||
#
WITH |
Did you mean one of the following? [ |
|
42703 |
#
UNRESOLVED |
Cannot resolve column |
#
WITHOUT |
||
#
WITH |
Otherwise did you mean one of the following column(s)? [ |
|
42703 |
#
UNRESOLVED |
USING column |
42704 |
#
AMBIGUOUS |
Reference |
42704 |
#
CANNOT |
Cannot resolve dataframe column |
42704 |
#
CANNOT |
Cannot resolve |
42704 |
#
CODEC |
Cannot find a short name for the codec |
42704 |
#
COLLATION |
The value |
42704 |
#
COLLATION |
The value |
42704 |
#
DATA |
Data source ' |
42704 |
#
DEFAULT |
Default database |
42704 |
#
ENCODER |
Not found an encoder of the type |
42704 |
#
FIELD |
No such struct field |
42704 |
#
INDEX |
Cannot find the index |
42704 |
#
SCHEMA |
The schema |
42704 |
#
UNRECOGNIZED |
Unrecognized SQL type - name: |
42710 |
#
ALTER |
ALTER TABLE |
42710 |
#
CREATE |
CREATE TABLE column |
42710 |
#
DATA |
Data source ' |
42710 |
#
DUPLICATED |
The metric name is not unique: |
42710 |
#
FIELD |
Cannot |
42710 |
#
FOUND |
Detected multiple data sources with the name ' |
42710 |
#
INDEX |
Cannot create the index |
42710 |
#
LOCATION |
Cannot name the managed table as |
42710 |
#
MULTIPLE |
Detected multiple data sources with the name |
42711 |
#
COLUMN |
The column |
42713 |
#
DUPLICATED |
Duplicated field names in Arrow Struct are not allowed, got |
42713 |
#
STATIC |
Static partition column |
42723 |
#
ROUTINE |
Cannot create the routine |
42723 |
#
VARIABLE |
Cannot create the variable |
4274K |
#
DUPLICATE |
Call to routine |
#
BOTH |
A positional argument and named argument both referred to the same parameter. Please remove the named argument referring to this parameter. |
|
#
DOUBLE |
More than one named argument referred to the same parameter. Please assign a value only once. |
|
4274K |
#
NAMED |
Named parameters are not supported for function |
4274K |
#
REQUIRED |
Cannot invoke function |
4274K |
#
UNEXPECTED |
Cannot invoke function |
4274K |
#
UNRECOGNIZED |
Cannot invoke function |
42802 |
#
ASSIGNMENT |
The number of columns or variables assigned or aliased: |
42802 |
#
STATEFUL |
Failed to perform stateful processor operation= |
42802 |
#
STATEFUL |
Failed to perform stateful processor operation= |
42802 |
#
STATEFUL |
Cannot re-initialize state on the same grouping key during initial state handling for stateful processor. Invalid grouping key= |
42802 |
#
STATEFUL |
Cannot use TTL for state= |
42802 |
#
STATEFUL |
TTL duration must be greater than zero for State store operation= |
42802 |
#
STATE |
Failed to create column family with unsupported starting character and name= |
42802 |
#
STATE |
Failed to perform column family operation= |
42802 |
#
STATE |
The handle has not been initialized for this StatefulProcessor. Please only use the StatefulProcessor within the transformWithState operator. |
42802 |
#
STATE |
Incorrect number of ordering ordinals= |
42802 |
#
STATE |
Incorrect number of prefix columns= |
42802 |
#
STATE |
Null type ordering column with name= |
42802 |
#
STATE |
State store operation= |
42802 |
#
STATE |
Variable size ordering column with name= |
42802 |
#
UDTF |
The number of aliases supplied in the AS clause does not match the number of columns output by the UDTF. Expected |
42802 |
#
UDTF |
Failed to evaluate the user-defined table function because its 'analyze' method returned a requested OrderingColumn whose column name expression included an unnecessary alias |
42802 |
#
UDTF |
Failed to evaluate the user-defined table function because its 'analyze' method returned a requested 'select' expression ( |
42803 |
#
GROUPING |
Column of grouping ( |
42803 |
#
GROUPING |
Columns of grouping_id ( |
42803 |
#
MISSING |
The non-aggregating expression |
42803 |
#
MISSING |
The query does not include a GROUP BY clause. Add GROUP BY or turn it into the window functions using OVER clauses. |
42803 |
#
UNRESOLVED |
Cannot infer grouping columns for GROUP BY ALL based on the select clause. Please explicitly specify the grouping columns. |
42805 |
#
GROUP |
GROUP BY position |
42805 |
#
ORDER |
ORDER BY position |
42809 |
#
EXPECT |
' |
42809 |
#
EXPECT |
' |
#
NO |
||
#
USE |
Please use ALTER VIEW instead. |
|
42809 |
#
EXPECT |
The table |
#
NO |
||
#
USE |
Please use ALTER TABLE instead. |
|
42809 |
#
FORBIDDEN |
The operation |
42809 |
#
NOT |
Operation |
42809 |
#
UNSUPPORTED |
Can't insert into the target. |
#
MULTI |
Can only write data to relations with a single path but given paths are |
|
#
NOT |
The target relation |
|
#
NOT |
The target relation |
|
#
RDD |
An RDD-based table is not allowed. |
|
#
READ |
The target relation |
|
42809 |
#
WRONG |
The operation |
42815 |
#
EMITTING |
Previous node emitted a row with eventTime= |
42818 |
#
INCOMPARABLE |
Invalid pivot column |
42822 |
#
EXPRESSION |
Column expression |
42822 |
#
GROUP |
The expression |
42823 |
#
INVALID |
Invalid subquery: |
#
SCALAR |
Scalar subquery must return only one column, but got |
|
42825 |
#
CANNOT |
Failed to merge incompatible data types |
42825 |
#
INCOMPATIBLE |
|
42826 |
#
NUM |
|
42826 |
#
NUM |
Number of given aliases does not match number of output columns. Function name: |
42845 |
#
AGGREGATE |
Non-deterministic expression |
42846 |
#
CANNOT |
Cannot cast |
42846 |
#
CANNOT |
Cannot convert Protobuf |
42846 |
#
CANNOT |
Unable to convert |
42846 |
#
CANNOT |
Cannot convert SQL |
42846 |
#
CANNOT |
Cannot convert SQL |
42846 |
#
CANNOT |
Cannot up cast |
42846 |
#
EXPRESSION |
Failed to decode a row to a value of the expressions: |
42846 |
#
EXPRESSION |
Failed to encode a value of the expressions: |
42846 |
#
UNEXPECTED |
The class |
42883 |
#
ROUTINE |
The routine |
42883 |
#
UNRESOLVABLE |
Could not resolve |
42883 |
#
UNRESOLVED |
Cannot resolve routine |
42883 |
#
UNRESOLVED |
Cannot resolve variable |
42883 |
#
VARIABLE |
The variable |
428C4 |
#
UNPIVOT |
All unpivot value columns must have the same size as there are value column names ( |
428EK |
#
TEMP |
CREATE TEMPORARY VIEW or the corresponding Dataset APIs only accept single-part view names, but got: |
428FR |
#
CANNOT |
ALTER TABLE (ALTER|CHANGE) COLUMN is not supported for partition columns, but found the partition column |
428FT |
#
PARTITIONS |
Cannot ADD or RENAME TO partition(s) |
428FT |
#
PARTITIONS |
The partition(s) |
428H2 |
#
EXCEPT |
EXCEPT column |
42902 |
#
UNSUPPORTED |
Can't overwrite the target that is also being read from. |
#
PATH
|
The target path is |
|
#
TABLE
|
The target table is |
|
42903 |
#
GROUP |
Aggregate functions are not allowed in GROUP BY, but found |
42903 |
#
GROUP |
GROUP BY |
42903 |
#
INVALID |
The FILTER expression |
#
CONTAINS |
Expected a FILTER expression without an aggregation, but found |
|
#
CONTAINS |
Expected a FILTER expression without a window function, but found |
|
#
NON |
Expected a deterministic FILTER expression. |
|
#
NOT |
Expected a FILTER expression of the BOOLEAN type. |
|
42903 |
#
INVALID |
The WHERE condition |
42908 |
#
SPECIFY |
Cannot specify both CLUSTER BY and CLUSTERED BY INTO BUCKETS. |
42908 |
#
SPECIFY |
Cannot specify both CLUSTER BY and PARTITIONED BY. |
429BB |
#
CANNOT |
Cannot recognize hive type string: |
42K01 |
#
DATATYPE |
DataType |
42K01 |
#
INCOMPLETE |
Incomplete complex type: |
#
ARRAY
|
The definition of "ARRAY" type is incomplete. You must provide an element type. For example: "ARRAY |
|
#
MAP
|
The definition of "MAP" type is incomplete. You must provide a key type and a value type. For example: "MAP |
|
#
STRUCT
|
The definition of "STRUCT" type is incomplete. You must provide at least one field type. For example: "STRUCT |
|
42K02 |
#
DATA |
Failed to find the data source: |
42K03 |
#
BATCH |
Unable to find batch |
42K03 |
#
CANNOT |
Could not load Protobuf class with name |
42K03 |
#
DATA |
The schema of the data source table does not match the expected schema. If you are using the DataFrameReader.schema API or creating a table, avoid specifying the schema. Data Source schema: |
42K03 |
#
LOAD |
LOAD DATA input path does not exist: |
42K03 |
#
PATH |
Path does not exist: |
42K03 |
#
RENAME |
Failed to rename as |
42K03 |
#
STDS |
Failed to read the state schema. Either the file does not exist, or the file is corrupted. options: |
42K03 |
#
STREAMING |
Streaming stateful operator name does not match with the operator in state metadata. This likely to happen when user adds/removes/changes stateful operator of existing streaming query. Stateful operators in the metadata: [ |
42K04 |
#
FAILED |
Failed to rename |
42K04 |
#
PATH |
Path |
42K05 |
#
INVALID |
The location name cannot be empty string, but |
42K05 |
#
REQUIRES |
|
42K06 |
#
INVALID |
Invalid options: |
#
NON |
Must use the |
|
#
NON |
A type of keys and values in |
|
42K07 |
#
INVALID |
The input schema |
#
NON |
The input expression must be string literal and not null. |
|
#
NON |
The input expression should be evaluated to struct type, but got |
|
#
PARSE |
Cannot parse the schema: |
|
42K08 |
#
INVALID |
The argument |
42K08 |
#
NON |
The function |
42K08 |
#
NON |
Literal expressions required for pivot values, found |
42K08 |
#
SEED |
The seed expression |
42K09 |
#
COMPLEX |
Cannot process input data types for the expression: |
#
MISMATCHED |
All input types must be the same except nullable, containsNull, valueContainsNull flags, but found the input types |
|
#
NO |
The collection of input data types must not be empty. |
|
42K09 |
#
DATATYPE |
Cannot resolve |
#
ARRAY |
Input to |
|
#
BINARY |
Input to function |
|
#
BINARY |
the left and right operands of the binary operator have incompatible types ( |
|
#
BINARY |
the binary operator requires the input type |
|
#
BLOOM |
The Bloom filter binary input to |
|
#
BLOOM |
Input to function |
|
#
CANNOT |
Unable to convert column |
|
#
CANNOT |
Cannot drop all fields in struct. |
|
#
CAST |
cannot cast |
|
#
CAST |
cannot cast |
|
#
CAST |
cannot cast |
|
#
CREATE |
The given keys of function |
|
#
CREATE |
The given values of function |
|
#
CREATE |
Only foldable |
|
#
DATA |
Input to |
|
#
FILTER |
Filter expression |
|
#
HASH |
Input to the function |
|
#
HASH |
Input to the function |
|
#
INPUT |
Length of |
|
#
INVALID |
The |
|
#
INVALID |
Input schema |
|
#
INVALID |
Input schema |
|
#
INVALID |
The key of map cannot be/contain |
|
#
INVALID |
The |
|
#
INVALID |
|
|
#
INVALID |
Input schema |
|
#
IN |
The data type of one or more elements in the left hand side of an IN subquery is not compatible with the data type of the output of the subquery. Mismatched columns: [ |
|
#
IN |
The number of columns in the left hand side of an IN subquery does not match the number of columns in the output of subquery. Left hand side columns(length: |
|
#
MAP |
The |
|
#
MAP |
Input to |
|
#
MAP |
Input to the |
|
#
NON |
the input |
|
#
NON |
all arguments must be strings. |
|
#
NULL |
Null typed values cannot be used as arguments of |
|
#
PARAMETER |
The |
|
#
RANGE |
The data type |
|
#
RANGE |
A range window frame with value boundaries cannot be used in a window specification with multiple order by expressions: |
|
#
RANGE |
A range window frame cannot be used in an unordered window specification. |
|
#
SEQUENCE |
|
|
#
SPECIFIED |
Window frame bounds |
|
#
SPECIFIED |
Window frame upper bound |
|
#
SPECIFIED |
The data type of the |
|
#
SPECIFIED |
Window frame |
|
#
SPECIFIED |
The lower bound of a window frame must be |
|
#
STACK |
The data type of the column ( |
|
#
TYPE |
|
|
#
UNEXPECTED |
class |
|
#
UNEXPECTED |
The |
|
#
UNEXPECTED |
The |
|
#
UNEXPECTED |
The |
|
#
UNEXPECTED |
cannot find a static method |
|
#
UNSUPPORTED |
The input of |
|
#
VALUE |
The |
|
#
WRONG |
The expression requires |
|
#
WRONG |
The number of endpoints must be >= 2 to construct intervals but the actual number is |
|
42K09 |
#
EVENT |
The event time |
42K09 |
#
INVALID |
Variable type must be string type but got |
42K09 |
#
PIVOT |
Invalid pivot value ' |
42K09 |
#
UNEXPECTED |
Parameter |
42K09 |
#
UNPIVOT |
Unpivot value columns must share a least common type, some types do not: [ |
42K0A |
#
UNPIVOT |
UNPIVOT requires all given |
42K0A |
#
UNPIVOT |
At least one value column needs to be specified for UNPIVOT, all columns specified as ids. |
42K0B |
#
INCONSISTENT |
You may get a different result due to the upgrading to |
#
DATETIME |
Spark >= 3.0: Fail to recognize |
|
#
DATETIME |
Spark >= 3.0: All week-based patterns are unsupported since Spark 3.0, detected week-based character: |
|
#
PARSE |
Spark >= 3.0: Fail to parse |
|
#
READ |
Spark >= 3.0: reading dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z from |
|
#
WRITE |
Spark >= 3.0: writing dates before 1582-10-15 or timestamps before 1900-01-01T00:00:00Z into |
|
42K0D |
#
INVALID |
Invalid lambda function call. |
#
DUPLICATE |
The lambda function has duplicate arguments |
|
#
NON |
A lambda function should only be used in a higher order function. However, its class is |
|
#
NUM |
A higher order function expects |
|
42K0E |
#
INVALID |
The limit like expression |
#
DATA |
The |
|
#
IS |
The |
|
#
IS |
The evaluated |
|
#
IS |
The |
|
42K0E |
#
INVALID |
The operator expects a deterministic expression, but the actual expression is |
42K0E |
#
INVALID |
Invalid observed metrics. |
#
AGGREGATE |
Aggregate expressions with DISTINCT are not allowed in observed metrics, but found: |
|
#
AGGREGATE |
Aggregate expression with FILTER predicate are not allowed in observed metrics, but found: |
|
#
MISSING |
The observed metrics should be named: |
|
#
NESTED |
Nested aggregates are not allowed in observed metrics, but found: |
|
#
NON |
Attribute |
|
#
NON |
Non-deterministic expression |
|
#
WINDOW |
Window expressions are not allowed in observed metrics, but found: |
|
42K0E |
#
INVALID |
Cannot specify both version and timestamp when time travelling the table. |
42K0E |
#
INVALID |
The time travel timestamp expression |
#
INPUT
|
Cannot be casted to the "TIMESTAMP" type. |
|
#
NON |
Must be deterministic. |
|
#
OPTION
|
Timestamp string in the options must be able to cast to TIMESTAMP type. |
|
#
UNEVALUABLE
|
Must be evaluable. |
|
42K0E |
#
JOIN |
The join condition |
42K0E |
#
MULTIPLE |
Cannot specify time travel in both the time travel clause and options. |
42K0E |
#
MULTI |
The expression |
42K0E |
#
NO |
df.mergeInto needs to be followed by at least one of whenMatched/whenNotMatched/whenNotMatchedBySource. |
42K0E |
#
UNSUPPORTED |
A query operator contains one or more unsupported expressions. Consider to rewrite it to avoid window functions, aggregate functions, and generator functions in the WHERE clause. Invalid expressions: [ |
42K0E |
#
UNSUPPORTED |
A query parameter contains unsupported expression. Parameters can either be variables or literals. Invalid expression: [ |
42K0E |
#
UNSUPPORTED |
The generator is not supported: |
#
MULTI |
only one generator allowed per SELECT clause but found |
|
#
NESTED |
nested in expressions |
|
#
NOT |
|
|
#
OUTSIDE |
outside the SELECT clause, found: |
|
42K0E |
#
UNSUPPORTED |
grouping()/grouping_id() can only be used with GroupingSets/Cube/Rollup. |
42K0E |
#
UNSUPPORTED |
MERGE operation contains unsupported |
#
AGGREGATE
|
Aggregates are not allowed: |
|
#
NON |
Non-deterministic expressions are not allowed: |
|
#
SUBQUERY
|
Subqueries are not allowed: |
|
42K0E |
#
UNTYPED |
You're using untyped Scala UDF, which does not have the input type information. Spark may blindly pass null to the Scala closure with primitive-type argument, and the closure will see the default value of the Java type for the null argument, e.g. |
42K0E |
#
WINDOW |
|
42K0F |
#
INVALID |
Cannot create the persistent object |
42K0G |
#
PROTOBUF |
Could not find dependency: |
42K0G |
#
PROTOBUF |
Error reading Protobuf descriptor file at path: |
42K0G |
#
PROTOBUF |
Searching for |
42K0G |
#
PROTOBUF |
Found |
42K0G |
#
PROTOBUF |
Type mismatch encountered for field: |
42K0G |
#
PROTOBUF |
Unable to locate Message |
42K0G |
#
PROTOBUF |
Protobuf type not yet supported: |
42K0G |
#
RECURSIVE |
Found recursive reference in Protobuf schema, which can not be processed by Spark by default: |
42K0G |
#
UNABLE |
Unable to convert SQL type |
42K0G |
#
UNKNOWN |
Attempting to treat |
42K0H |
#
RECURSIVE |
Recursive view |
42K0I |
#
SQL |
The SQL config |
42K0J |
#
UNSET |
Attempted to unset non-existent properties [ |
42K0K |
#
INVALID |
Invalid inverse distribution function |
#
DISTINCT |
Cannot use DISTINCT with WITHIN GROUP. |
|
#
WITHIN |
WITHIN GROUP is required for inverse distribution function. |
|
#
WRONG |
Requires |
|
42KD0 |
#
AMBIGUOUS |
Name |
42KD9 |
#
CANNOT |
Failed merging schemas: Initial schema: |
42KD9 |
#
UNABLE |
Unable to infer schema for |
42KDE |
#
CALL |
The method |
42KDE |
#
CANNOT |
Failed to create data source table |
#
EXTERNAL |
provider ' |
|
42KDE |
#
INVALID |
The data source writer has generated an invalid number of commit messages. Expected exactly one writer commit message from each task, but received |
42KDE |
#
NON |
Window function is not supported in |
42KDF |
#
XML |
|
42P01 |
#
TABLE |
The table or view |
42P01 |
#
VIEW |
The view |
42P02 |
#
UNBOUND |
Found the unbound parameter: |
42P06 |
#
SCHEMA |
Cannot create schema |
42P07 |
#
TABLE |
Cannot create table or view |
42P07 |
#
TEMP |
Cannot create the temporary view |
42P07 |
#
VIEW |
Cannot create view |
42P08 |
#
CATALOG |
The catalog |
42P20 |
#
UNSUPPORTED |
Expression |
42P21 |
#
COLLATION |
Could not determine which collation to use for string functions and operators. |
#
EXPLICIT
|
Error occurred due to the mismatch between explicit collations: |
|
#
IMPLICIT
|
Error occurred due to the mismatch between multiple implicit non-default collations. Use COLLATE function to set the collation explicitly. |
|
42P22 |
#
INDETERMINATE |
Function called requires knowledge of the collation it should apply, but indeterminate collation was found. Use COLLATE function to set the collation explicitly. |
42S22 |
#
NO |
Cannot find |
46103 |
#
CANNOT |
Cannot load class |
46110 |
#
CANNOT |
Cannot modify the value of the Spark config: |
46121 |
#
INVALID |
The datasource |
51024 |
#
INCOMPATIBLE |
The SQL query of view |
53200 |
#
UNABLE |
Unable to acquire |
54000 |
#
COLLECTION |
Can't create array with |
#
FUNCTION
|
unsuccessful try to create arrays in the function |
|
#
INITIALIZE
|
cannot initialize an array with specified parameters. |
|
#
PARAMETER
|
the value of parameter(s) |
|
54000 |
#
GROUPING |
Grouping sets size cannot be greater than |
54001 |
#
FAILED |
The statement, including potential SQL functions and referenced views, was too complex to parse. To mitigate this error divide the statement into multiple, less complex chunks. |
54006 |
#
EXCEED |
Exceeds char/varchar type length limitation: |
54006 |
#
KRYO |
Kryo serialization failed: |
54023 |
#
TABLE |
There are too many table arguments for table-valued function. It allows one table argument, but got: |
54K00 |
#
VIEW |
The depth of view |
56000 |
#
CHECKPOINT |
Checkpoint block |
56038 |
#
CODEC |
The codec |
#
WITH |
Available codecs are |
|
#
WITH |
Consider to set the config |
|
56038 |
#
FEATURE |
The feature |
56038 |
#
GET |
Hive 2.2 and lower versions don't support getTablesByType. Please use Hive 2.3 or higher version. |
56038 |
#
INCOMPATIBLE |
Detected an incompatible DataSourceRegister. Please remove the incompatible library from classpath or upgrade it. Error: |
56K00 |
#
CONNECT
|
Generic Spark Connect error. |
#
INTERCEPTOR |
Cannot instantiate GRPC interceptor because |
|
#
INTERCEPTOR |
Error instantiating GRPC interceptor: |
|
#
PLUGIN |
Cannot instantiate Spark Connect plugin because |
|
#
PLUGIN |
Error instantiating Spark Connect plugin: |
|
#
SESSION |
Both Datasets must belong to the same SparkSession. |
|
58030 |
#
CANNOT |
An error occurred during loading state. |
#
CANNOT |
Cannot read RocksDB checkpoint metadata. Expected |
|
#
CANNOT |
Error reading delta file |
|
#
CANNOT |
Error reading delta file |
|
#
CANNOT |
Error reading snapshot file |
|
#
CANNOT |
Error reading snapshot file |
|
#
CANNOT |
Error reading streaming state file of |
|
#
UNCATEGORIZED
|
||
#
UNEXPECTED |
Copied |
|
#
UNEXPECTED |
Version cannot be |
|
#
UNRELEASED |
|
|
58030 |
#
CANNOT |
Failed to set permissions on created path |
58030 |
#
CANNOT |
Error writing state store files for provider |
#
CANNOT |
Cannot perform commit during state checkpoint. |
|
58030 |
#
FAILED |
Failed to rename temp file |
58030 |
#
INVALID |
Invalid bucket file: |
58030 |
#
TASK |
Task failed while writing rows to |
58030 |
#
UNABLE |
Unable to fetch tables of Hive database: |
F0000 |
#
INVALID |
System memory |
F0000 |
#
INVALID |
Executor memory |
F0000 |
#
INVALID |
The value of the config " |
HV000 |
#
FAILED |
Failed JDBC |
#
ALTER |
Alter the table |
|
#
CREATE |
Create the index |
|
#
CREATE |
Create the namespace |
|
#
CREATE |
Create a comment on the namespace: |
|
#
CREATE |
Create the table |
|
#
DROP |
Drop the index |
|
#
DROP |
Drop the namespace |
|
#
GET |
Get tables from the namespace: |
|
#
LIST |
List namespaces. |
|
#
NAMESPACE |
Check that the namespace |
|
#
REMOVE |
Remove a comment on the namespace: |
|
#
RENAME |
Rename the table |
|
#
TABLE |
Check that the table |
|
#
UNCLASSIFIED
|
|
|
HV091 |
#
NONEXISTENT |
Field(s) |
HY000 |
#
INVALID |
The handle |
#
FORMAT
|
Handle must be an UUID string of the format '00112233-4455-6677-8899-aabbccddeeff' |
|
#
OPERATION |
Operation was considered abandoned because of inactivity and removed. |
|
#
OPERATION |
Operation already exists. |
|
#
OPERATION |
Operation not found. |
|
#
SESSION |
The existing Spark server driver instance has restarted. Please reconnect. |
|
#
SESSION |
Session was closed. |
|
#
SESSION |
Session not found. |
|
HY008 |
#
OPERATION |
Operation has been canceled. |
HY109 |
#
INVALID |
The cursor is invalid. |
#
DISCONNECTED
|
The cursor has been disconnected by the server. |
|
#
NOT |
The cursor is not reattachable. |
|
#
POSITION |
The cursor position id |
|
#
POSITION |
The cursor position id |
|
KD000 |
#
FAILED |
Failed to register classes with Kryo. |
KD000 |
#
GRAPHITE |
Invalid Graphite protocol: |
KD000 |
#
GRAPHITE |
Graphite sink requires ' |
KD000 |
#
INCOMPATIBLE |
Cannot write incompatible data for the table |
#
AMBIGUOUS |
Ambiguous column name in the input data |
|
#
CANNOT |
Cannot find data for the output column |
|
#
CANNOT |
Cannot safely cast |
|
#
EXTRA |
Cannot write extra columns |
|
#
EXTRA |
Cannot write extra fields |
|
#
NULLABLE |
Cannot write nullable elements to array of non-nulls: |
|
#
NULLABLE |
Cannot write nullable values to non-null column |
|
#
NULLABLE |
Cannot write nullable values to map of non-nulls: |
|
#
STRUCT |
Struct |
|
#
UNEXPECTED |
Struct |
|
KD000 |
#
MALFORMED |
Malformed CSV record: |
KD001 |
#
FAILED |
Encountered error while reading file |
#
CANNOT |
Could not read footer. Please ensure that the file is in either ORC or Parquet format. If not, please convert it to a valid format. If the file is in the valid format, please check if it is corrupt. If it is, you can choose to either ignore it or fix the corruption. |
|
#
FILE |
File does not exist. It is possible the underlying files have been updated. You can explicitly invalidate the cache in Spark by running 'REFRESH TABLE tableName' command in SQL or by recreating the Dataset/DataFrame involved. |
|
#
NO |
||
#
PARQUET |
Data type mismatches when reading Parquet column |
|
KD005 |
#
ALL |
Cannot use all columns for partition columns. |
KD006 |
#
STDS |
No committed batch found, checkpoint location: |
KD006 |
#
STDS |
The state does not have any partition. Please double check that the query points to the valid state. options: |
KD006 |
#
STDS |
The offset log for |
KD006 |
#
STDS |
Metadata is not available for offset log for |
KD00B |
#
ERROR |
Error reading avro data -- encountered an unknown fingerprint: |
P0001 |
#
USER |
|
P0001 |
#
USER |
The |
P0001 |
#
USER |
The |
XX000 |
#
INVALID |
The view |
XX000 |
#
MALFORMED |
Malformed Protobuf messages are detected in message deserialization. Parse Mode: |
XX000 |
#
MISSING |
Resolved attribute(s) |
#
RESOLVED |
Attribute(s) with the same name appear in the operation: |
|
#
RESOLVED |
||
XXKD0 |
#
PLAN |
The input plan of |
XXKD0 |
#
PLAN |
Rule |
XXKDA |
#
SPARK |
Job |
XXKST |
#
STATE |
|
XXKST |
#
STATE |
Binary inequality column is not supported with state store. Provided schema: |
XXKST |
#
STDS |
Internal error: |
XXKST |
#
STREAMING |
Streaming Runner initialization failed, returned |
XXKST |
#
STREAM |
Query [id = |
XXKUC |
#
INSUFFICIENT |
Can't find table property: |
#
MISSING |
|
|
#
MISSING |
|