[feat](mysql-compatibility) Increase compatibility with mysql (#7041)

Increase compatibility with mysql
  1. Added two system tables files and partitions
  2. Improved the return logic of mysql error code to make the error code more compatible with mysql
  3. Added lock/unlock tables statement and show columns statement for compatibility with mysql dump
  4. Compatible with mysqldump tool, now you can use mysql dump to dump data and table structure from doris

now use mysqldump may print error message like 
```
$ mysqldump -h127.0.0.1 -P9130 -uroot test_query_qa > a
mysqldump: Error: 'errCode = 2, detailMessage = select list expression not produced by aggregation output (missing from GROUP BY clause?): `EXTRA`' when trying to dump tablespaces
```

This error message not effect the export file, you can add `--no-tablespaces` to avoid this error
This commit is contained in:
Zhengguo Yang
2021-11-20 21:39:37 +08:00
committed by GitHub
parent e9282205f1
commit 52ebb3d8f5
45 changed files with 2428 additions and 249 deletions

View File

@ -88,6 +88,8 @@ set(EXEC_FILES
schema_scanner/schema_table_privileges_scanner.cpp
schema_scanner/schema_schema_privileges_scanner.cpp
schema_scanner/schema_user_privileges_scanner.cpp
schema_scanner/schema_files_scanner.cpp
schema_scanner/schema_partitions_scanner.cpp
partitioned_hash_table.cc
partitioned_hash_table_ir.cc

View File

@ -170,8 +170,10 @@ Status SchemaScanNode::prepare(RuntimeState* state) {
}
if (_src_tuple_desc->slots()[j]->type().type != _dest_tuple_desc->slots()[i]->type().type) {
LOG(WARNING) << "schema not match. input is " << _src_tuple_desc->slots()[j]->type()
<< " and output is " << _dest_tuple_desc->slots()[i]->type();
LOG(WARNING) << "schema not match. input is " << _src_tuple_desc->slots()[j]->col_name()
<< "(" << _src_tuple_desc->slots()[j]->type() << ") and output is "
<< _dest_tuple_desc->slots()[i]->col_name() << "("
<< _dest_tuple_desc->slots()[i]->type() << ")";
return Status::InternalError("schema not match.");
}
_index_map[i] = j;

View File

@ -21,6 +21,8 @@
#include "exec/schema_scanner/schema_collations_scanner.h"
#include "exec/schema_scanner/schema_columns_scanner.h"
#include "exec/schema_scanner/schema_dummy_scanner.h"
#include "exec/schema_scanner/schema_files_scanner.h"
#include "exec/schema_scanner/schema_partitions_scanner.h"
#include "exec/schema_scanner/schema_schema_privileges_scanner.h"
#include "exec/schema_scanner/schema_schemata_scanner.h"
#include "exec/schema_scanner/schema_statistics_scanner.h"
@ -105,6 +107,10 @@ SchemaScanner* SchemaScanner::create(TSchemaTableType::type type) {
return new (std::nothrow) SchemaSchemaPrivilegesScanner();
case TSchemaTableType::SCH_USER_PRIVILEGES:
return new (std::nothrow) SchemaUserPrivilegesScanner();
case TSchemaTableType::SCH_FILES:
return new (std::nothrow) SchemaFilesScanner();
case TSchemaTableType::SCH_PARTITIONS:
return new (std::nothrow) SchemaPartitionsScanner();
default:
return new (std::nothrow) SchemaDummyScanner();
break;

View File

@ -0,0 +1,115 @@
// 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.
#include "exec/schema_scanner/schema_files_scanner.h"
#include "exec/schema_scanner/schema_helper.h"
#include "runtime/primitive_type.h"
#include "runtime/string_value.h"
namespace doris {
SchemaScanner::ColumnDesc SchemaFilesScanner::_s_tbls_columns[] = {
// name, type, size, is_null
{"FILE_ID", TYPE_BIGINT, sizeof(int64_t), true},
{"FILE_NAME", TYPE_STRING, sizeof(StringValue), true},
{"FILE_TYPE", TYPE_VARCHAR, sizeof(StringValue), true},
{"TABLESPACE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
{"TABLE_CATALOG", TYPE_CHAR, sizeof(StringValue), false},
{"TABLE_SCHEMA", TYPE_STRING, sizeof(StringValue), true},
{"TABLE_NAME", TYPE_STRING, sizeof(StringValue), true},
{"LOGFILE_GROUP_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
{"LOGFILE_GROUP_NUMBER", TYPE_BIGINT, sizeof(int64_t), true},
{"ENGINE", TYPE_VARCHAR, sizeof(StringValue), false},
{"FULLTEXT_KEYS", TYPE_STRING, sizeof(StringValue), true},
{"DELETED_ROWS", TYPE_STRING, sizeof(StringValue), true},
{"UPDATE_COUNT", TYPE_STRING, sizeof(StringValue), true},
{"FREE_EXTENTS", TYPE_BIGINT, sizeof(int64_t), true},
{"TOTAL_EXTENTS", TYPE_BIGINT, sizeof(int64_t), true},
{"EXTENT_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
{"INITIAL_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
{"MAXIMUM_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
{"AUTOEXTEND_SIZE", TYPE_BIGINT, sizeof(int64_t), true},
{"CREATION_TIME", TYPE_STRING, sizeof(StringValue), true},
{"LAST_UPDATE_TIME", TYPE_STRING, sizeof(StringValue), true},
{"LAST_ACCESS_TIME", TYPE_STRING, sizeof(StringValue), true},
{"RECOVER_TIME", TYPE_STRING, sizeof(StringValue), true},
{"TRANSACTION_COUNTER", TYPE_STRING, sizeof(StringValue), true},
{"VERSION", TYPE_BIGINT, sizeof(int64_t), true},
{"ROW_FORMAT", TYPE_VARCHAR, sizeof(StringValue), true},
{"TABLE_ROWS", TYPE_STRING, sizeof(StringValue), true},
{"AVG_ROW_LENGTH", TYPE_STRING, sizeof(StringValue), true},
{"DATA_LENGTH", TYPE_STRING, sizeof(StringValue), true},
{"MAX_DATA_LENGTH", TYPE_STRING, sizeof(StringValue), true},
{"INDEX_LENGTH", TYPE_STRING, sizeof(StringValue), true},
{"DATA_FREE", TYPE_BIGINT, sizeof(int64_t), true},
{"CREATE_TIME", TYPE_STRING, sizeof(StringValue), true},
{"UPDATE_TIME", TYPE_STRING, sizeof(StringValue), true},
{"CHECK_TIME", TYPE_STRING, sizeof(StringValue), true},
{"CHECKSUM", TYPE_STRING, sizeof(StringValue), true},
{"STATUS", TYPE_VARCHAR, sizeof(StringValue), true},
{"EXTRA", TYPE_VARCHAR, sizeof(StringValue), true},
};
SchemaFilesScanner::SchemaFilesScanner()
: SchemaScanner(_s_tbls_columns,
sizeof(_s_tbls_columns) / sizeof(SchemaScanner::ColumnDesc)),
_db_index(0),
_table_index(0) {}
SchemaFilesScanner::~SchemaFilesScanner() {}
Status SchemaFilesScanner::start(RuntimeState* state) {
if (!_is_init) {
return Status::InternalError("used before initialized.");
}
TGetDbsParams db_params;
if (NULL != _param->db) {
db_params.__set_pattern(*(_param->db));
}
if (NULL != _param->current_user_ident) {
db_params.__set_current_user_ident(*(_param->current_user_ident));
} else {
if (NULL != _param->user) {
db_params.__set_user(*(_param->user));
}
if (NULL != _param->user_ip) {
db_params.__set_user_ip(*(_param->user_ip));
}
}
if (NULL != _param->ip && 0 != _param->port) {
RETURN_IF_ERROR(
SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result));
} else {
return Status::InternalError("IP or port doesn't exists");
}
return Status::OK();
}
Status SchemaFilesScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos) {
if (!_is_init) {
return Status::InternalError("Used before initialized.");
}
if (nullptr == tuple || nullptr == pool || nullptr == eos) {
return Status::InternalError("input pointer is nullptr.");
}
*eos = true;
return Status::OK();
}
} // namespace doris

View File

@ -0,0 +1,43 @@
// 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.
#ifndef DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_FILES_SCANNER_H
#define DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_FILES_SCANNER_H
#include "exec/schema_scanner.h"
#include "gen_cpp/FrontendService_types.h"
namespace doris {
class SchemaFilesScanner : public SchemaScanner {
public:
SchemaFilesScanner();
virtual ~SchemaFilesScanner();
virtual Status start(RuntimeState* state);
virtual Status get_next_row(Tuple* tuple, MemPool* pool, bool* eos);
int _db_index;
int _table_index;
TGetDbsResult _db_result;
TListTableStatusResult _table_result;
static SchemaScanner::ColumnDesc _s_tbls_columns[];
};
} // namespace doris
#endif

View File

@ -0,0 +1,103 @@
// 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.
#include "exec/schema_scanner/schema_partitions_scanner.h"
#include "exec/schema_scanner/schema_helper.h"
#include "runtime/datetime_value.h"
#include "runtime/primitive_type.h"
#include "runtime/string_value.h"
namespace doris {
SchemaScanner::ColumnDesc SchemaPartitionsScanner::_s_tbls_columns[] = {
// name, type, size, is_null
{"TABLE_CATALOG", TYPE_VARCHAR, sizeof(StringValue), true},
{"TABLE_SCHEMA", TYPE_VARCHAR, sizeof(StringValue), true},
{"TABLE_NAME", TYPE_VARCHAR, sizeof(StringValue), false},
{"PARTITION_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
{"SUBPARTITION_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
{"PARTITION_ORDINAL_POSITION", TYPE_INT, sizeof(int32_t), true},
{"SUBPARTITION_ORDINAL_POSITION", TYPE_INT, sizeof(int32_t), true},
{"PARTITION_METHOD", TYPE_VARCHAR, sizeof(StringValue), true},
{"SUBPARTITION_METHOD", TYPE_VARCHAR, sizeof(StringValue), true},
{"PARTITION_EXPRESSION", TYPE_VARCHAR, sizeof(StringValue), true},
{"SUBPARTITION_EXPRESSION", TYPE_VARCHAR, sizeof(StringValue), true},
{"PARTITION_DESCRIPTION", TYPE_STRING, sizeof(StringValue), true},
{"TABLE_ROWS", TYPE_BIGINT, sizeof(int64_t), true},
{"AVG_ROW_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
{"DATA_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
{"MAX_DATA_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
{"INDEX_LENGTH", TYPE_BIGINT, sizeof(int64_t), true},
{"DATA_FREE", TYPE_BIGINT, sizeof(int64_t), true},
{"CREATE_TIME", TYPE_BIGINT, sizeof(int64_t), false},
{"UPDATE_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
{"CHECK_TIME", TYPE_DATETIME, sizeof(DateTimeValue), true},
{"CHECKSUM", TYPE_BIGINT, sizeof(int64_t), true},
{"PARTITION_COMMENT", TYPE_STRING, sizeof(StringValue), false},
{"NODEGROUP", TYPE_VARCHAR, sizeof(StringValue), true},
{"TABLESPACE_NAME", TYPE_VARCHAR, sizeof(StringValue), true},
};
SchemaPartitionsScanner::SchemaPartitionsScanner()
: SchemaScanner(_s_tbls_columns,
sizeof(_s_tbls_columns) / sizeof(SchemaScanner::ColumnDesc)),
_db_index(0),
_table_index(0) {}
SchemaPartitionsScanner::~SchemaPartitionsScanner() {}
Status SchemaPartitionsScanner::start(RuntimeState* state) {
if (!_is_init) {
return Status::InternalError("used before initialized.");
}
TGetDbsParams db_params;
if (NULL != _param->db) {
db_params.__set_pattern(*(_param->db));
}
if (NULL != _param->current_user_ident) {
db_params.__set_current_user_ident(*(_param->current_user_ident));
} else {
if (NULL != _param->user) {
db_params.__set_user(*(_param->user));
}
if (NULL != _param->user_ip) {
db_params.__set_user_ip(*(_param->user_ip));
}
}
if (NULL != _param->ip && 0 != _param->port) {
RETURN_IF_ERROR(
SchemaHelper::get_db_names(*(_param->ip), _param->port, db_params, &_db_result));
} else {
return Status::InternalError("IP or port doesn't exists");
}
return Status::OK();
}
Status SchemaPartitionsScanner::get_next_row(Tuple* tuple, MemPool* pool, bool* eos) {
if (!_is_init) {
return Status::InternalError("Used before initialized.");
}
if (nullptr == tuple || nullptr == pool || nullptr == eos) {
return Status::InternalError("input pointer is nullptr.");
}
*eos = true;
return Status::OK();
}
} // namespace doris

View File

@ -0,0 +1,43 @@
// 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.
#ifndef DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_PARTITIONS_SCANNER_H
#define DORIS_BE_SRC_QUERY_EXEC_SCHEMA_SCANNER_SCHEMA_PARTITIONS_SCANNER_H
#include "exec/schema_scanner.h"
#include "gen_cpp/FrontendService_types.h"
namespace doris {
class SchemaPartitionsScanner : public SchemaScanner {
public:
SchemaPartitionsScanner();
virtual ~SchemaPartitionsScanner();
virtual Status start(RuntimeState* state);
virtual Status get_next_row(Tuple* tuple, MemPool* pool, bool* eos);
int _db_index;
int _table_index;
TGetDbsResult _db_result;
TListTableStatusResult _table_result;
static SchemaScanner::ColumnDesc _s_tbls_columns[];
};
} // namespace doris
#endif

View File

@ -20,8 +20,6 @@
#include "exec/schema_scanner/schema_helper.h"
#include "runtime/primitive_type.h"
#include "runtime/string_value.h"
//#include "runtime/datetime_value.h"
namespace doris {
SchemaScanner::ColumnDesc SchemaTablesScanner::_s_tbls_columns[] = {

View File

@ -241,8 +241,8 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALIAS, KW_ALL, KW_A
KW_DATA, KW_DATABASE, KW_DATABASES, KW_DATE, KW_DATETIME, KW_DAY, KW_DECIMAL, KW_DECOMMISSION, KW_DEFAULT, KW_DESC, KW_DESCRIBE,
KW_DELETE, KW_UPDATE, KW_DISTINCT, KW_DISTINCTPC, KW_DISTINCTPCSA, KW_DISTRIBUTED, KW_DISTRIBUTION, KW_DYNAMIC, KW_BUCKETS, KW_DIV, KW_DOUBLE, KW_DROP, KW_DROPP, KW_DUPLICATE,
KW_ELSE, KW_ENABLE, KW_ENCRYPTKEY, KW_ENCRYPTKEYS, KW_END, KW_ENGINE, KW_ENGINES, KW_ENTER, KW_ERRORS, KW_EVENTS, KW_EXCEPT, KW_EXCLUDE,
KW_EXISTS, KW_EXPORT, KW_EXTERNAL, KW_EXTRACT,
KW_FALSE, KW_FEATURE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FILE, KW_FILTER, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORCE, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, KW_FUNCTIONS,
KW_EXISTS, KW_EXPORT, KW_EXTENDED, KW_EXTERNAL, KW_EXTRACT,
KW_FALSE, KW_FEATURE, KW_FOLLOWER, KW_FOLLOWING, KW_FREE, KW_FROM, KW_FIELDS, KW_FILE, KW_FILTER, KW_FIRST, KW_FLOAT, KW_FOR, KW_FORCE, KW_FORMAT, KW_FRONTEND, KW_FRONTENDS, KW_FULL, KW_FUNCTION, KW_FUNCTIONS,
KW_GLOBAL, KW_GRANT, KW_GRANTS, KW_GRAPH, KW_GROUP, KW_GROUPING,
KW_HASH, KW_HAVING, KW_HDFS, KW_HELP,KW_HLL, KW_HLL_UNION, KW_HOUR, KW_HUB,
KW_IDENTIFIED, KW_IF, KW_IN, KW_INDEX, KW_INDEXES, KW_INFILE, KW_INSTALL,
@ -250,7 +250,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALIAS, KW_ALL, KW_A
KW_JOB, KW_JOIN,
KW_KEY, KW_KEYS, KW_KILL,
KW_LABEL, KW_LARGEINT, KW_LAST, KW_LEFT, KW_LESS, KW_LEVEL, KW_LIKE, KW_LIMIT, KW_LINK, KW_LIST, KW_LOAD,
KW_LOCAL, KW_LOCATION, KW_LATERAL,
KW_LOCAL, KW_LOCATION, KW_LOCK, KW_LOW_PRIORITY, KW_LATERAL,
KW_MAP, KW_MATERIALIZED, KW_MAX, KW_MAX_VALUE, KW_MERGE, KW_MIN, KW_MINUTE, KW_MINUS, KW_MIGRATE, KW_MIGRATIONS, KW_MODIFY, KW_MONTH,
KW_NAME, KW_NAMES, KW_NEGATIVE, KW_NO, KW_NOT, KW_NULL, KW_NULLS,
KW_OBSERVER, KW_OFFSET, KW_ON, KW_ONLY, KW_OPEN, KW_OR, KW_ORDER, KW_OUTER, KW_OUTFILE, KW_OVER,
@ -267,7 +267,7 @@ terminal String KW_ADD, KW_ADMIN, KW_AFTER, KW_AGGREGATE, KW_ALIAS, KW_ALL, KW_A
KW_SUM, KW_SUPERUSER, KW_SYNC, KW_SYSTEM,
KW_TABLE, KW_TABLES, KW_TABLET, KW_TASK, KW_TEMPORARY, KW_TERMINATED, KW_TEXT, KW_THAN, KW_TIME, KW_THEN, KW_TIMESTAMP, KW_TINYINT,KW_TRASH,
KW_TO, KW_TRANSACTION, KW_TRIGGERS, KW_TRIM, KW_TRUE, KW_TRUNCATE, KW_TYPE, KW_TYPES,
KW_UNCOMMITTED, KW_UNBOUNDED, KW_UNION, KW_UNIQUE, KW_UNSIGNED, KW_USE, KW_USER, KW_USING, KW_UNINSTALL,
KW_UNCOMMITTED, KW_UNBOUNDED, KW_UNION, KW_UNIQUE, KW_UNLOCK, KW_UNSIGNED, KW_USE, KW_USER, KW_USING, KW_UNINSTALL,
KW_VALUE, KW_VALUES, KW_VARCHAR, KW_VARIABLES, KW_VERBOSE, KW_VIEW,
KW_WARNINGS, KW_WEEK, KW_WHEN, KW_WHITELIST, KW_WHERE, KW_WITH, KW_WORK, KW_WRITE,
KW_YEAR, KW_SQL_BLOCK_RULE;
@ -295,7 +295,7 @@ nonterminal StatementBase stmt, show_stmt, show_param, help_stmt, load_stmt,
use_stmt, kill_stmt, drop_stmt, recover_stmt, grant_stmt, revoke_stmt, create_stmt, set_stmt, sync_stmt, cancel_stmt, cancel_param, delete_stmt,
link_stmt, migrate_stmt, enter_stmt, transaction_stmt, unsupported_stmt, export_stmt, admin_stmt, truncate_stmt,
import_columns_stmt, import_delete_on_stmt, import_sequence_stmt, import_where_stmt, install_plugin_stmt, uninstall_plugin_stmt,
import_preceding_filter_stmt;
import_preceding_filter_stmt, unlock_tables_stmt, lock_tables_stmt;
nonterminal String transaction_label;
nonterminal ImportColumnDesc import_column_desc;
@ -346,7 +346,7 @@ nonterminal Expr set_expr_or_default;
nonterminal ArrayList<Expr> expr_list, values, row_value, opt_values;
nonterminal ArrayList<Expr> func_arg_list;
nonterminal ArrayList<Expr> expr_pipe_list;
nonterminal String select_alias, opt_table_alias;
nonterminal String select_alias, opt_table_alias, lock_alias;
nonterminal ArrayList<String> ident_list;
nonterminal PartitionNames opt_partition_names, partition_names;
nonterminal ClusterName cluster_name;
@ -517,6 +517,9 @@ nonterminal Boolean opt_signed_unsigned;
nonterminal StorageBackend storage_backend;
nonterminal ArrayList<LockTable> opt_lock_tables_list;
nonterminal LockTable lock_table;
precedence nonassoc COMMA;
precedence nonassoc STRING_LITERAL;
precedence nonassoc KW_COLUMNS;
@ -726,6 +729,10 @@ stmt ::=
{: RESULT = stmt; :}
| uninstall_plugin_stmt : stmt
{: RESULT = stmt; :}
| lock_tables_stmt:stmt
{: RESULT = stmt; :}
| unlock_tables_stmt:stmt
{: RESULT = stmt; :}
| /* empty: query only has comments */
{:
RESULT = new EmptyStmt();
@ -2997,10 +3004,26 @@ describe_stmt ::=
{:
RESULT = new DescribeStmt(table, false);
:}
| KW_SHOW KW_FIELDS KW_FROM table_name:table
{:
RESULT = new DescribeStmt(table, false);
:}
| KW_SHOW KW_COLUMNS KW_FROM table_name:table
{:
RESULT = new DescribeStmt(table, false);
:}
| describe_command table_name:table KW_ALL
{:
RESULT = new DescribeStmt(table, true);
:}
| KW_SHOW KW_EXTENDED KW_FIELDS KW_FROM table_name:table
{:
RESULT = new DescribeStmt(table, true);
:}
| KW_SHOW KW_EXTENDED KW_COLUMNS KW_FROM table_name:table
{:
RESULT = new DescribeStmt(table, true);
:}
| describe_command opt_explain_options:options query_stmt:query
{:
query.setIsExplain(options);
@ -3343,6 +3366,74 @@ backup_stmt ::=
:}
;
unlock_tables_stmt ::=
KW_UNLOCK KW_TABLES
{:
RESULT = new UnlockTablesStmt();
:}
;
lock_alias ::=
/* empty */
{:
RESULT = null;
:}
| KW_AS ident:ident
{:
RESULT = ident;
:}
| KW_AS STRING_LITERAL:l
{:
RESULT = l;
:}
;
lock_table ::=
table_name:name lock_alias:alias KW_READ
{:
RESULT = new LockTable(name, alias, LockTable.LockType.READ);
:}
|
table_name:name lock_alias:alias KW_READ KW_LOCAL
{:
RESULT = new LockTable(name, alias, LockTable.LockType.READ_LOCAL);
:}
|
table_name:name lock_alias:alias KW_WRITE
{:
RESULT = new LockTable(name, alias, LockTable.LockType.WRITE);
:}
|
table_name:name lock_alias:alias KW_LOW_PRIORITY KW_WRITE
{:
RESULT = new LockTable(name, alias, LockTable.LockType.LOW_PRIORITY_WRITE);
:}
;
opt_lock_tables_list ::=
lock_table: table
{:
ArrayList<LockTable> lock_tables = new ArrayList<LockTable>();
lock_tables.add(table);
RESULT = lock_tables;
:}
| opt_lock_tables_list:lock_tables COMMA lock_table:table
{:
lock_tables.add(table);
RESULT = lock_tables;
:}
|
{:
RESULT = new ArrayList<LockTable>();
:}
;
lock_tables_stmt ::=
KW_LOCK KW_TABLES opt_lock_tables_list:lock_tables
{:
RESULT = new LockTablesStmt(lock_tables);
:}
;
opt_backup_table_ref_list ::=
backup_exclude_or_not:isExclude LPAREN base_table_ref_list:tbls RPAREN
{:
@ -5277,6 +5368,8 @@ keyword ::=
{: RESULT = id; :}
| KW_FILE:id
{: RESULT = id; :}
| KW_FIELDS:id
{: RESULT = id; :}
| KW_FILTER:id
{: RESULT = id; :}
| KW_FIRST:id
@ -5341,6 +5434,10 @@ keyword ::=
{: RESULT = id; :}
| KW_LOCATION:id
{: RESULT = id; :}
| KW_LOCK:id
{: RESULT = id; :}
| KW_UNLOCK:id
{: RESULT = id; :}
| KW_MATERIALIZED:id
{: RESULT = id; :}
| KW_MERGE:id

View File

@ -63,7 +63,7 @@ public class AlterDatabaseQuotaStmt extends DdlStmt {
super.analyze(analyzer);
if (!Catalog.getCurrentCatalog().getAuth().checkGlobalPriv(ConnectContext.get(), PrivPredicate.ADMIN)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), dbName);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), dbName);
}
if (Strings.isNullOrEmpty(dbName)) {

View File

@ -60,7 +60,7 @@ public class AlterDatabaseRename extends DdlStmt {
PrivPredicate.of(PrivBitSet.of(PaloPrivilege.ADMIN_PRIV,
PaloPrivilege.ALTER_PRIV),
Operator.OR))) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), dbName);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), dbName);
}
if (Strings.isNullOrEmpty(newDbName)) {

View File

@ -57,7 +57,7 @@ public class CreateDbStmt extends DdlStmt {
dbName = ClusterNamespace.getFullName(getClusterName(), dbName);
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.CREATE)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), dbName);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), dbName);
}
}

View File

@ -61,11 +61,11 @@ public class DropDbStmt extends DdlStmt {
dbName = ClusterNamespace.getFullName(getClusterName(), dbName);
// Don't allowed to drop 'information_schema'
if (dbName.equalsIgnoreCase(ClusterNamespace.getFullName(getClusterName(), InfoSchemaDb.DATABASE_NAME))) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), dbName);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), dbName);
}
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.DROP)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED,
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR,
ConnectContext.get().getQualifiedUser(), dbName);
}
}

View File

@ -0,0 +1,62 @@
// 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.doris.analysis;
public class LockTable {
public enum LockType {
READ("READ"),
READ_LOCAL("READ LOCAL"),
WRITE("WRITE"),
LOW_PRIORITY_WRITE("LOW_PRIORITY WRITE");
private String desc;
LockType(String description) {
this.desc = description;
}
@Override
public String toString() {
return desc;
}
}
private TableName tableName;
private String alias;
private LockType lockType;
public LockTable(TableName tableName, String alias, LockType lockType) {
this.tableName = tableName;
this.alias = alias;
this.lockType = lockType;
}
public LockTable(TableName tableName, LockType lockType) {
this(tableName, null, lockType);
}
public TableName getTableName() {
return tableName;
}
public String getAlias() {
return alias;
}
public LockType getLockType() {
return lockType;
}
}

View File

@ -0,0 +1,100 @@
// 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.doris.analysis;
import org.apache.doris.catalog.Catalog;
import org.apache.doris.catalog.Database;
import org.apache.doris.catalog.Table;
import org.apache.doris.cluster.ClusterNamespace;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ErrorReport;
import org.apache.doris.common.UserException;
import org.apache.doris.mysql.privilege.PrivPredicate;
import org.apache.doris.qe.ConnectContext;
import com.google.common.base.Strings;
import org.apache.logging.log4j.LogManager;
import org.apache.logging.log4j.Logger;
import java.util.ArrayList;
import java.util.List;
public class LockTablesStmt extends StatementBase {
private static final Logger LOG = LogManager.getLogger(LockTablesStmt.class);
private List<LockTable> lockTables;
public LockTablesStmt(ArrayList<LockTable> lockTables) {
this.lockTables = lockTables;
}
@Override
public void analyze(Analyzer analyzer) throws UserException {
super.analyze(analyzer);
for (LockTable lockTable : lockTables) {
String dbName = lockTable.getTableName().getDb();
String tableName = lockTable.getTableName().getTbl();
if (Strings.isNullOrEmpty(dbName)) {
dbName = analyzer.getDefaultDb();
} else {
dbName = ClusterNamespace.getFullName(analyzer.getClusterName(), dbName);
}
if (Strings.isNullOrEmpty(dbName)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
}
if (Strings.isNullOrEmpty(tableName)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName);
}
Database db = analyzer.getCatalog().getDbOrAnalysisException(dbName);
Table table = db.getTableOrAnalysisException(tableName);
// check auth
if (!Catalog.getCurrentCatalog().getAuth().checkTblPriv(ConnectContext.get(), dbName,
tableName,
PrivPredicate.SELECT)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_TABLEACCESS_DENIED_ERROR, "SELECT",
ConnectContext.get().getQualifiedUser(),
ConnectContext.get().getRemoteIP(),
tableName);
}
}
}
@Override
public String toSql() {
StringBuilder sb = new StringBuilder();
sb.append("LOCK TABLES ");
for (int i = 0; i < lockTables.size(); i++) {
if (i != 0) {
sb.append(", ");
}
sb.append(lockTables.get(i).getTableName().toSql());
if (lockTables.get(i).getAlias() != null) {
sb.append(" AS ").append(lockTables.get(i).getAlias());
}
sb.append(" ").append(lockTables.get(i).getLockType().toString());
}
return sb.toString();
}
@Override
public RedirectStatus getRedirectStatus() {
return RedirectStatus.NO_FORWARD;
}
}

View File

@ -55,7 +55,7 @@ public class RecoverDbStmt extends DdlStmt {
PaloPrivilege.CREATE_PRIV,
PaloPrivilege.ADMIN_PRIV),
Operator.OR))) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), dbName);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), dbName);
}
}

View File

@ -299,7 +299,7 @@ public class SelectStmt extends QueryStmt {
ErrorReport.reportAnalysisException(ErrorCode.ERR_NO_DB_ERROR);
}
if (Strings.isNullOrEmpty(tableName)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR);
ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName);
}
Database db = analyzer.getCatalog().getDbOrAnalysisException(dbName);
Table table = db.getTableOrAnalysisException(tableName);
@ -861,7 +861,7 @@ public class SelectStmt extends QueryStmt {
private void expandStar(Analyzer analyzer, TableName tblName) throws AnalysisException {
Collection<TupleDescriptor> descs = analyzer.getDescriptor(tblName);
if (descs == null || descs.isEmpty()) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, tblName.getTbl());
ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_TABLE, tblName.getTbl(), tblName.getDb());
}
for (TupleDescriptor desc : descs) {
expandStar(tblName, desc);

View File

@ -71,7 +71,7 @@ public class ShowBackupStmt extends ShowStmt {
// check auth
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.LOAD)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED,
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR,
ConnectContext.get().getQualifiedUser(), dbName);
}

View File

@ -68,7 +68,7 @@ public class ShowCreateDbStmt extends ShowStmt {
PaloPrivilege.CREATE_PRIV,
PaloPrivilege.DROP_PRIV),
Operator.OR))) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED,
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR,
ConnectContext.get().getQualifiedUser(), db);
}
}

View File

@ -76,7 +76,7 @@ public class ShowCreateFunctionStmt extends ShowStmt {
// check operation privilege
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.SHOW)) {
ErrorReport.reportAnalysisException(
ErrorCode.ERR_DB_ACCESS_DENIED, ConnectContext.get().getQualifiedUser(), dbName);
ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName);
}
// analyze arguments
argsDef.analyze(analyzer);

View File

@ -68,7 +68,7 @@ public class ShowEncryptKeysStmt extends ShowStmt{
// must check after analyze dbName, for case dbName is null.
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.ADMIN)) {
ErrorReport.reportAnalysisException(
ErrorCode.ERR_DB_ACCESS_DENIED, ConnectContext.get().getQualifiedUser(), dbName);
ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName);
}
}

View File

@ -96,7 +96,7 @@ public class ShowFunctionsStmt extends ShowStmt {
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.SHOW)) {
ErrorReport.reportAnalysisException(
ErrorCode.ERR_DB_ACCESS_DENIED, ConnectContext.get().getQualifiedUser(), dbName);
ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName);
}
if (expr != null) {

View File

@ -77,7 +77,7 @@ public class ShowRestoreStmt extends ShowStmt {
// check auth
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.LOAD)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED,
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR,
ConnectContext.get().getQualifiedUser(), dbName);
}

View File

@ -64,7 +64,7 @@ public class ShowSmallFilesStmt extends ShowStmt {
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), dbName, PrivPredicate.SHOW)) {
ErrorReport.reportAnalysisException(
ErrorCode.ERR_DB_ACCESS_DENIED, ConnectContext.get().getQualifiedUser(), dbName);
ErrorCode.ERR_DBACCESS_DENIED_ERROR, ConnectContext.get().getQualifiedUser(), dbName);
}
}

View File

@ -88,7 +88,7 @@ public class ShowTableStatusStmt extends ShowStmt {
db = ClusterNamespace.getFullName(analyzer.getClusterName(), db);
}
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), db, PrivPredicate.SHOW)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), db);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), db);
}
}

View File

@ -0,0 +1,30 @@
// 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.doris.analysis;
public class UnlockTablesStmt extends StatementBase {
@Override
public String toSql() {
return "UNLOCK TABLES";
}
@Override
public RedirectStatus getRedirectStatus() {
return RedirectStatus.NO_FORWARD;
}
}

View File

@ -64,7 +64,7 @@ public class UseStmt extends StatementBase {
database = ClusterNamespace.getFullName(getClusterName(), database);
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), database, PrivPredicate.SHOW)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED, analyzer.getQualifiedUser(), database);
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, analyzer.getQualifiedUser(), database);
}
}

View File

@ -2862,7 +2862,7 @@ public class Catalog {
ErrorReport.reportDdlException(ErrorCode.ERR_TABLE_EXISTS_ERROR, tableName);
}
if (!Catalog.getCurrentRecycleBin().recoverTable(db, tableName)) {
ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName);
}
} finally {
db.writeUnlock();
@ -4313,7 +4313,6 @@ public class Catalog {
sb.append(new PrintableMap<>(hiveTable.getHiveProperties(), " = ", true, true, false).toString());
sb.append("\n)");
}
sb.append(";");
createTableStmt.add(sb.toString());
@ -4518,7 +4517,7 @@ public class Catalog {
LOG.info("drop table[{}] which does not exist", tableName);
return;
} else {
ErrorReport.reportDdlException(ErrorCode.ERR_BAD_TABLE_ERROR, tableName);
ErrorReport.reportDdlException(ErrorCode.ERR_UNKNOWN_TABLE, tableName, dbName);
}
}
// Check if a view
@ -4793,27 +4792,33 @@ public class Catalog {
}
public Database getDbOrMetaException(String dbName) throws MetaNotFoundException {
return getDbOrException(dbName, s -> new MetaNotFoundException("unknown databases, dbName=" + s));
return getDbOrException(dbName, s -> new MetaNotFoundException("unknown databases, dbName=" + s,
ErrorCode.ERR_BAD_DB_ERROR));
}
public Database getDbOrMetaException(long dbId) throws MetaNotFoundException {
return getDbOrException(dbId, s -> new MetaNotFoundException("unknown databases, dbId=" + s));
return getDbOrException(dbId, s -> new MetaNotFoundException("unknown databases, dbId=" + s,
ErrorCode.ERR_BAD_DB_ERROR));
}
public Database getDbOrDdlException(String dbName) throws DdlException {
return getDbOrException(dbName, s -> new DdlException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
return getDbOrException(dbName, s -> new DdlException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s),
ErrorCode.ERR_BAD_DB_ERROR));
}
public Database getDbOrDdlException(long dbId) throws DdlException {
return getDbOrException(dbId, s -> new DdlException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
return getDbOrException(dbId, s -> new DdlException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s),
ErrorCode.ERR_BAD_DB_ERROR));
}
public Database getDbOrAnalysisException(String dbName) throws AnalysisException {
return getDbOrException(dbName, s -> new AnalysisException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
return getDbOrException(dbName, s -> new AnalysisException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s),
ErrorCode.ERR_BAD_DB_ERROR));
}
public Database getDbOrAnalysisException(long dbId) throws AnalysisException {
return getDbOrException(dbId, s -> new AnalysisException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s)));
return getDbOrException(dbId, s -> new AnalysisException(ErrorCode.ERR_BAD_DB_ERROR.formatErrorMsg(s),
ErrorCode.ERR_BAD_DB_ERROR));
}
public EditLog getEditLog() {
@ -5816,7 +5821,7 @@ public class Catalog {
// Change current database of this session.
public void changeDb(ConnectContext ctx, String qualifiedDb) throws DdlException {
if (!auth.checkDbPriv(ctx, qualifiedDb, PrivPredicate.SHOW)) {
ErrorReport.reportDdlException(ErrorCode.ERR_DB_ACCESS_DENIED, ctx.getQualifiedUser(), qualifiedDb);
ErrorReport.reportDdlException(ErrorCode.ERR_DBACCESS_DENIED_ERROR, ctx.getQualifiedUser(), qualifiedDb);
}
this.getDbOrDdlException(qualifiedDb);

View File

@ -522,7 +522,8 @@ public class Database extends MetaObject implements Writable {
}
public Table getTableOrAnalysisException(String tableName) throws AnalysisException {
return getTableOrException(tableName, t -> new AnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR.formatErrorMsg(t)));
return getTableOrException(tableName, t -> new AnalysisException(ErrorCode.ERR_UNKNOWN_TABLE.formatErrorMsg(t
, fullQualifiedName)));
}
public OlapTable getOlapTableOrAnalysisException(String tableName) throws AnalysisException {

View File

@ -338,6 +338,85 @@ public class SchemaTable extends Table {
.column("INDEX_TYPE", ScalarType.createVarchar(16))
.column("COMMENT", ScalarType.createVarchar(16))
.build()))
.put("files",
new SchemaTable(
SystemIdGenerator.getNextId(),
"files",
TableType.SCHEMA,
builder()
.column("FILE_ID", ScalarType.createType(PrimitiveType.BIGINT))
.column("FILE_NAME", ScalarType.createStringType())
.column("FILE_TYPE", ScalarType.createVarchar(256))
.column("TABLESPACE_NAME", ScalarType.createVarchar(256))
.column("TABLE_CATALOG", ScalarType.createCharType(16))
.column("TABLE_SCHEMA", ScalarType.createStringType())
.column("TABLE_NAME", ScalarType.createStringType())
.column("LOGFILE_GROUP_NAME", ScalarType.createVarchar(256))
.column("LOGFILE_GROUP_NUMBER", ScalarType.createType(PrimitiveType.BIGINT))
.column("ENGINE", ScalarType.createVarchar(64))
.column("FULLTEXT_KEYS", ScalarType.createStringType())
.column("DELETED_ROWS", ScalarType.createStringType())
.column("UPDATE_COUNT", ScalarType.createStringType())
.column("FREE_EXTENTS", ScalarType.createType(PrimitiveType.BIGINT))
.column("TOTAL_EXTENTS", ScalarType.createType(PrimitiveType.BIGINT))
.column("EXTENT_SIZE", ScalarType.createType(PrimitiveType.BIGINT))
.column("INITIAL_SIZE", ScalarType.createType(PrimitiveType.BIGINT))
.column("MAXIMUM_SIZE", ScalarType.createType(PrimitiveType.BIGINT))
.column("AUTOEXTEND_SIZE", ScalarType.createType(PrimitiveType.BIGINT))
.column("CREATION_TIME", ScalarType.createStringType())
.column("LAST_UPDATE_TIME", ScalarType.createStringType())
.column("LAST_ACCESS_TIME", ScalarType.createStringType())
.column("RECOVER_TIME", ScalarType.createStringType())
.column("TRANSACTION_COUNTER", ScalarType.createStringType())
.column("VERSION", ScalarType.createType(PrimitiveType.BIGINT))
.column("ROW_FORMAT", ScalarType.createVarchar(256))
.column("TABLE_ROWS", ScalarType.createStringType())
.column("AVG_ROW_LENGTH", ScalarType.createStringType())
.column("DATA_LENGTH", ScalarType.createStringType())
.column("MAX_DATA_LENGTH", ScalarType.createStringType())
.column("INDEX_LENGTH", ScalarType.createStringType())
.column("DATA_FREE", ScalarType.createType(PrimitiveType.BIGINT))
.column("CREATE_TIME", ScalarType.createStringType())
.column("UPDATE_TIME", ScalarType.createStringType())
.column("CHECK_TIME", ScalarType.createStringType())
.column("CHECKSUM", ScalarType.createStringType())
.column("STATUS", ScalarType.createVarchar(256))
.column("EXTRA", ScalarType.createVarchar(256))
.build()))
.put("partitions",
new SchemaTable(
SystemIdGenerator.getNextId(),
"partitions",
TableType.SCHEMA,
builder()
.column("TABLE_CATALOG", ScalarType.createVarchar(64))
.column("TABLE_SCHEMA", ScalarType.createVarchar(64))
.column("TABLE_NAME", ScalarType.createVarchar(64))
.column("PARTITION_NAME", ScalarType.createVarchar(64))
.column("SUBPARTITION_NAME", ScalarType.createVarchar(64))
.column("PARTITION_ORDINAL_POSITION",
ScalarType.createType(PrimitiveType.INT))
.column("SUBPARTITION_ORDINAL_POSITION",
ScalarType.createType(PrimitiveType.INT))
.column("PARTITION_METHOD", ScalarType.createVarchar(13))
.column("SUBPARTITION_METHOD", ScalarType.createVarchar(13))
.column("PARTITION_EXPRESSION", ScalarType.createVarchar(2048))
.column("SUBPARTITION_EXPRESSION", ScalarType.createVarchar(2048))
.column("PARTITION_DESCRIPTION", ScalarType.createStringType())
.column("TABLE_ROWS", ScalarType.createType(PrimitiveType.BIGINT))
.column("AVG_ROW_LENGTH", ScalarType.createType(PrimitiveType.BIGINT))
.column("DATA_LENGTH", ScalarType.createType(PrimitiveType.BIGINT))
.column("MAX_DATA_LENGTH", ScalarType.createType(PrimitiveType.BIGINT))
.column("INDEX_LENGTH", ScalarType.createType(PrimitiveType.BIGINT))
.column("DATA_FREE", ScalarType.createType(PrimitiveType.BIGINT))
.column("CREATE_TIME", ScalarType.createType(PrimitiveType.BIGINT))
.column("UPDATE_TIME", ScalarType.createType(PrimitiveType.DATETIME))
.column("CHECK_TIME", ScalarType.createType(PrimitiveType.DATETIME))
.column("CHECKSUM", ScalarType.createType(PrimitiveType.BIGINT))
.column("PARTITION_COMMENT", ScalarType.createStringType())
.column("NODEGROUP", ScalarType.createVarchar(256))
.column("TABLESPACE_NAME", ScalarType.createVarchar(268))
.build()))
.build();
// statistics is table provides information about table indexes in mysql: 5.7
// views column is from show create table views in mysql: 5.5.6

View File

@ -29,4 +29,9 @@ public class AnalysisException extends UserException {
public AnalysisException(String msg) {
super(msg);
}
public AnalysisException(String msg, ErrorCode mysqlErrorCode) {
super(msg);
setMysqlErrorCode(mysqlErrorCode);
}
}

View File

@ -21,6 +21,10 @@ public class DdlException extends UserException {
public DdlException(String msg) {
super(msg);
}
public DdlException(String msg, ErrorCode mysqlErrorCode) {
super(msg);
setMysqlErrorCode(mysqlErrorCode);
}
public DdlException(String msg, Throwable e) {
super(msg, e);

File diff suppressed because it is too large Load Diff

View File

@ -39,7 +39,7 @@ public class ErrorReport {
public static void reportAnalysisException(String pattern, Object... objs)
throws AnalysisException {
throw new AnalysisException(reportCommon(pattern, ErrorCode.ERR_UNKNOWN_ERROR, objs));
throw new AnalysisException(reportCommon(pattern, ErrorCode.ERR_UNKNOWN_ERROR, objs), ErrorCode.ERR_UNKNOWN_ERROR);
}
public static void reportAnalysisException(ErrorCode errorCode, Object... objs)
@ -49,7 +49,7 @@ public class ErrorReport {
public static void reportAnalysisException(String pattern, ErrorCode errorCode, Object... objs)
throws AnalysisException {
throw new AnalysisException(reportCommon(pattern, errorCode, objs));
throw new AnalysisException(reportCommon(pattern, errorCode, objs), errorCode);
}
public static void reportDdlException(String pattern, Object... objs)

View File

@ -25,6 +25,11 @@ public class MetaNotFoundException extends UserException {
super(InternalErrorCode.META_NOT_FOUND_ERR, msg);
}
public MetaNotFoundException(String msg, ErrorCode mysqlErrorCode) {
super(InternalErrorCode.META_NOT_FOUND_ERR, msg);
setMysqlErrorCode(mysqlErrorCode);
}
public MetaNotFoundException(InternalErrorCode errcode, String msg) {
super(errcode, msg);
}

View File

@ -24,35 +24,50 @@ import com.google.common.base.Strings;
*/
public class UserException extends Exception {
private InternalErrorCode errorCode;
private ErrorCode mysqlErrorCode;
public UserException(String msg, Throwable cause) {
super(Strings.nullToEmpty(msg), cause);
errorCode = InternalErrorCode.INTERNAL_ERR;
mysqlErrorCode = ErrorCode.ERR_UNKNOWN_ERROR;
}
public UserException(Throwable cause) {
super(cause);
errorCode = InternalErrorCode.INTERNAL_ERR;
mysqlErrorCode = ErrorCode.ERR_UNKNOWN_ERROR;
}
public UserException(String msg, Throwable cause, boolean enableSuppression, boolean writableStackTrace) {
super(Strings.nullToEmpty(msg), cause, enableSuppression, writableStackTrace);
errorCode = InternalErrorCode.INTERNAL_ERR;
mysqlErrorCode = ErrorCode.ERR_UNKNOWN_ERROR;
}
public UserException(String msg) {
super(Strings.nullToEmpty(msg));
errorCode = InternalErrorCode.INTERNAL_ERR;
mysqlErrorCode = ErrorCode.ERR_UNKNOWN_ERROR;
}
public UserException(InternalErrorCode errCode, String msg) {
super(Strings.nullToEmpty(msg));
this.errorCode = errCode;
mysqlErrorCode = ErrorCode.ERR_UNKNOWN_ERROR;
}
public InternalErrorCode getErrorCode() {
return errorCode;
}
public ErrorCode getMysqlErrorCode() {
return mysqlErrorCode;
}
public void setMysqlErrorCode(ErrorCode mysqlErrorCode) {
this.mysqlErrorCode = mysqlErrorCode;
}
@Override
public String getMessage() {
return errorCode + ", detailMessage = " + super.getMessage();

View File

@ -17,6 +17,7 @@
package org.apache.doris.mysql.nio;
import org.apache.doris.catalog.Catalog;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.mysql.MysqlProto;
import org.apache.doris.qe.ConnectContext;
import org.apache.doris.qe.ConnectProcessor;
@ -67,7 +68,7 @@ public class AcceptListener implements ChannelListener<AcceptingChannel<StreamCo
MysqlProto.sendResponsePacket(context);
connection.setCloseListener(streamConnection -> connectScheduler.unregisterConnection(context));
} else {
context.getState().setError("Reach limit of connections");
context.getState().setError(ErrorCode.ERR_TOO_MANY_USER_CONNECTIONS, "Reach limit of connections");
MysqlProto.sendResponsePacket(context);
throw new AfterConnectedException("Reach limit of connections");
}

View File

@ -84,7 +84,7 @@ public class ConnectProcessor {
private void handleInitDb() {
String dbName = new String(packetBuf.array(), 1, packetBuf.limit() - 1);
if (Strings.isNullOrEmpty(ctx.getClusterName())) {
ctx.getState().setError("Please enter cluster");
ctx.getState().setError(ErrorCode.ERR_CLUSTER_NAME_NULL, "Please enter cluster");
return;
}
dbName = ClusterNamespace.getFullName(ctx.getClusterName(), dbName);
@ -177,7 +177,7 @@ public class ConnectProcessor {
} catch (UnsupportedEncodingException e) {
// impossible
LOG.error("UTF8 is not supported in this environment.");
ctx.getState().setError("Unsupported character set(UTF-8)");
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_CHARACTER_SET, "Unsupported character set(UTF-8)");
return;
}
String sqlHash = DigestUtils.md5Hex(originStmt);
@ -186,7 +186,7 @@ public class ConnectProcessor {
Catalog.getCurrentCatalog().getSqlBlockRuleMgr().matchSql(originStmt, sqlHash, ctx.getQualifiedUser());
} catch (AnalysisException e) {
LOG.warn(e.getMessage());
ctx.getState().setError(e.getMessage());
ctx.getState().setError(e.getMysqlErrorCode(), e.getMessage());
return;
}
ctx.getAuditEventBuilder().reset();
@ -226,17 +226,18 @@ public class ConnectProcessor {
} catch (IOException e) {
// Client failed.
LOG.warn("Process one query failed because IOException: ", e);
ctx.getState().setError("Doris process failed");
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Doris process failed");
} catch (UserException e) {
LOG.warn("Process one query failed because.", e);
ctx.getState().setError(e.getMessage());
ctx.getState().setError(e.getMysqlErrorCode(), e.getMessage());
// set is as ANALYSIS_ERR so that it won't be treated as a query failure.
ctx.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
} catch (Throwable e) {
// Catch all throwable.
// If reach here, maybe palo bug.
LOG.warn("Process one query failed because unknown reason: ", e);
ctx.getState().setError(e.getClass().getSimpleName() + ", msg: " + e.getMessage());
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR,
e.getClass().getSimpleName() + ", msg: " + e.getMessage());
if (parsedStmt instanceof KillStmt) {
// ignore kill stmt execute err(not monitor it)
ctx.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
@ -296,17 +297,17 @@ public class ConnectProcessor {
return;
}
if (Strings.isNullOrEmpty(tableName)) {
ctx.getState().setError("Empty tableName");
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_TABLE, "Empty tableName");
return;
}
Database db = ctx.getCatalog().getDbNullable(ctx.getDatabase());
if (db == null) {
ctx.getState().setError("Unknown database(" + ctx.getDatabase() + ")");
ctx.getState().setError(ErrorCode.ERR_BAD_DB_ERROR, "Unknown database(" + ctx.getDatabase() + ")");
return;
}
Table table = db.getTableNullable(tableName);
if (table == null) {
ctx.getState().setError("Unknown table(" + tableName + ")");
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_TABLE, "Unknown table(" + tableName + ")");
return;
}
@ -335,7 +336,7 @@ public class ConnectProcessor {
MysqlCommand command = MysqlCommand.fromCode(code);
if (command == null) {
ErrorReport.report(ErrorCode.ERR_UNKNOWN_COM_ERROR);
ctx.getState().setError("Unknown command(" + command + ")");
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_COM_ERROR, "Unknown command(" + command + ")");
LOG.warn("Unknown command(" + command + ")");
return;
}
@ -359,7 +360,7 @@ public class ConnectProcessor {
handlePing();
break;
default:
ctx.getState().setError("Unsupported command(" + command + ")");
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_COM_ERROR, "Unsupported command(" + command + ")");
LOG.warn("Unsupported command(" + command + ")");
break;
}
@ -480,7 +481,9 @@ public class ConnectProcessor {
// so ctx.getCurrentUserIdentity() will get null, and causing NullPointerException after using it.
// return error directly.
TMasterOpResult result = new TMasterOpResult();
ctx.getState().setError("Missing current user identity. You need to upgrade this Frontend to the same version as Master Frontend.");
ctx.getState().setError(ErrorCode.ERR_ACCESS_DENIED_ERROR, "Missing current user identity. You need to upgrade this Frontend " +
"to the " +
"same version as Master Frontend.");
result.setMaxJournalId(Catalog.getCurrentCatalog().getMaxJournalId().longValue());
result.setPacket(getResultPacket());
return result;
@ -503,12 +506,12 @@ public class ConnectProcessor {
} catch (IOException e) {
// Client failed.
LOG.warn("Process one query failed because IOException: ", e);
ctx.getState().setError("Doris process failed: " + e.getMessage());
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Doris process failed: " + e.getMessage());
} catch (Throwable e) {
// Catch all throwable.
// If reach here, maybe Doris bug.
LOG.warn("Process one query failed because unknown reason: ", e);
ctx.getState().setError("Unexpected exception: " + e.getMessage());
ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Unexpected exception: " + e.getMessage());
}
// no matter the master execute success or fail, the master must transfer the result to follower
// and tell the follower the current journalID.

View File

@ -19,6 +19,7 @@ package org.apache.doris.qe;
import org.apache.doris.catalog.Catalog;
import org.apache.doris.common.Config;
import org.apache.doris.common.ErrorCode;
import org.apache.doris.common.ThreadPoolManager;
import org.apache.doris.ldap.LdapAuthenticate;
import org.apache.doris.mysql.MysqlProto;
@ -173,7 +174,7 @@ public class ConnectScheduler {
if (registerConnection(context)) {
MysqlProto.sendResponsePacket(context);
} else {
context.getState().setError("Reach limit of connections");
context.getState().setError(ErrorCode.ERR_USER_LIMIT_REACHED, "Reach limit of connections");
MysqlProto.sendResponsePacket(context);
return;
}

View File

@ -165,6 +165,8 @@ public class SessionVariable implements Serializable, Writable {
public static final String ENABLE_LATERAL_VIEW = "enable_lateral_view";
public static final String SQL_QUOTE_SHOW_CREATE = "sql_quote_show_create";
// session origin value
public Map<Field, String> sessionOriginValue = new HashMap<Field, String>();
// check stmt is or not [select /*+ SET_VAR(...)*/ ...]
@ -390,6 +392,9 @@ public class SessionVariable implements Serializable, Writable {
@VariableMgr.VarAttr(name = DISABLE_JOIN_REORDER)
private boolean disableJoinReorder = false;
@VariableMgr.VarAttr(name = SQL_QUOTE_SHOW_CREATE)
public boolean sqlQuoteShowCreate = true;
public long getMaxExecMemByte() {
return maxExecMemByte;
}
@ -520,6 +525,13 @@ public class SessionVariable implements Serializable, Writable {
}
}
public boolean isSqlQuoteShowCreate() {
return sqlQuoteShowCreate;
}
public void setSqlQuoteShowCreate(boolean sqlQuoteShowCreate) {
this.sqlQuoteShowCreate = sqlQuoteShowCreate;
}
public void setLoadMemLimit(long loadMemLimit) {
this.loadMemLimit = loadMemLimit;
}

View File

@ -1110,7 +1110,7 @@ public class ShowExecutor {
// forward compatibility
if (!Catalog.getCurrentCatalog().getAuth().checkDbPriv(ConnectContext.get(), db.getFullName(),
PrivPredicate.SHOW)) {
ErrorReport.reportAnalysisException(ErrorCode.ERR_DB_ACCESS_DENIED,
ErrorReport.reportAnalysisException(ErrorCode.ERR_DBACCESS_DENIED_ERROR,
ConnectContext.get().getQualifiedUser(),
db.getFullName());
}
@ -1483,7 +1483,8 @@ public class ShowExecutor {
Long id = olapTable.getIndexIdByName(indexName);
if (id == null) {
// invalid indexName
ErrorReport.reportAnalysisException(ErrorCode.ERR_BAD_TABLE_ERROR, showStmt.getIndexName());
ErrorReport.reportAnalysisException(ErrorCode.ERR_UNKNOWN_TABLE, showStmt.getIndexName(),
showStmt.getDbName());
}
indexId = id;
}

View File

@ -27,6 +27,7 @@ import org.apache.doris.analysis.Expr;
import org.apache.doris.analysis.InsertStmt;
import org.apache.doris.analysis.KillStmt;
import org.apache.doris.analysis.LiteralExpr;
import org.apache.doris.analysis.LockTablesStmt;
import org.apache.doris.analysis.NullLiteral;
import org.apache.doris.analysis.OutFileClause;
import org.apache.doris.analysis.QueryStmt;
@ -45,6 +46,7 @@ import org.apache.doris.analysis.TransactionBeginStmt;
import org.apache.doris.analysis.TransactionCommitStmt;
import org.apache.doris.analysis.TransactionRollbackStmt;
import org.apache.doris.analysis.TransactionStmt;
import org.apache.doris.analysis.UnlockTablesStmt;
import org.apache.doris.analysis.UnsupportedStmt;
import org.apache.doris.analysis.UseStmt;
import org.apache.doris.catalog.Catalog;
@ -398,25 +400,30 @@ public class StmtExecutor implements ProfileWriter {
handleKill();
} else if (parsedStmt instanceof ExportStmt) {
handleExportStmt();
} else if (parsedStmt instanceof UnlockTablesStmt) {
handleUnlockTablesStmt();
} else if (parsedStmt instanceof LockTablesStmt) {
handleLockTablesStmt();
} else if (parsedStmt instanceof UnsupportedStmt) {
handleUnsupportedStmt();
} else {
context.getState().setError("Do not support this query.");
context.getState().setError(ErrorCode.ERR_NOT_SUPPORTED_YET, "Do not support this query.");
}
} catch (IOException e) {
LOG.warn("execute IOException ", e);
// the exception happens when interact with client
// this exception shows the connection is gone
context.getState().setError(e.getMessage());
context.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, e.getMessage());
throw e;
} catch (UserException e) {
// analysis exception only print message, not print the stack
LOG.warn("execute Exception. {}", e.getMessage());
context.getState().setError(e.getMessage());
context.getState().setError(e.getMysqlErrorCode(), e.getMessage());
context.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
} catch (Exception e) {
LOG.warn("execute Exception", e);
context.getState().setError(e.getClass().getSimpleName() + ", msg: " + e.getMessage());
context.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR,
e.getClass().getSimpleName() + ", msg: " + e.getMessage());
if (parsedStmt instanceof KillStmt) {
// ignore kill stmt execute err(not monitor it)
context.getState().setErrType(QueryState.ErrType.ANALYSIS_ERR);
@ -431,7 +438,7 @@ public class StmtExecutor implements ProfileWriter {
sessionVariable.clearSessionOriginValue();
} catch (DdlException e) {
LOG.warn("failed to revert Session value.", e);
context.getState().setError(e.getMessage());
context.getState().setError(e.getMysqlErrorCode(), e.getMessage());
}
if (!context.isTxnModel() && parsedStmt instanceof InsertStmt) {
InsertStmt insertStmt = (InsertStmt) parsedStmt;
@ -701,7 +708,7 @@ public class StmtExecutor implements ProfileWriter {
executor.execute();
} catch (DdlException e) {
// Return error message to client.
context.getState().setError(e.getMessage());
context.getState().setError(ErrorCode.ERR_LOCAL_VARIABLE, e.getMessage());
return;
}
context.getState().setOk();
@ -1224,7 +1231,8 @@ public class StmtExecutor implements ProfileWriter {
// if in strict mode, insert will fail if there are filtered rows
if (context.getSessionVariable().getEnableInsertStrict()) {
if (filteredRows > 0) {
context.getState().setError("Insert has filtered data in strict mode, tracking_url="
context.getState().setError(ErrorCode.ERR_FAILED_WHEN_INSERT, "Insert has filtered data in strict mode, " +
"tracking_url="
+ coord.getTrackingUrl());
return;
}
@ -1271,7 +1279,7 @@ public class StmtExecutor implements ProfileWriter {
if (!Strings.isNullOrEmpty(coord.getTrackingUrl())) {
sb.append(". url: " + coord.getTrackingUrl());
}
context.getState().setError(sb.toString());
context.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, sb.toString());
return;
}
@ -1342,7 +1350,7 @@ public class StmtExecutor implements ProfileWriter {
}
context.getCatalog().changeDb(context, useStmt.getDatabase());
} catch (DdlException e) {
context.getState().setError(e.getMessage());
context.getState().setError(e.getMysqlErrorCode(), e.getMessage());
return;
}
context.getState().setOk();
@ -1420,6 +1428,10 @@ public class StmtExecutor implements ProfileWriter {
sendResult(resultSet);
}
private void handleUnlockTablesStmt() {
}
private void handleLockTablesStmt() {
}
private void handleExplainStmt(String result) throws IOException {
ShowResultSetMetaData metaData =
ShowResultSetMetaData.builder()
@ -1445,11 +1457,11 @@ public class StmtExecutor implements ProfileWriter {
} catch (UserException e) {
// Return message to info client what happened.
LOG.debug("DDL statement({}) process failed.", originStmt.originStmt, e);
context.getState().setError(e.getMessage());
context.getState().setError(e.getMysqlErrorCode(), e.getMessage());
} catch (Exception e) {
// Maybe our bug
LOG.warn("DDL statement(" + originStmt.originStmt + ") process failed.", e);
context.getState().setError("Unexpected exception: " + e.getMessage());
context.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR, "Unexpected exception: " + e.getMessage());
}
}
@ -1460,7 +1472,7 @@ public class StmtExecutor implements ProfileWriter {
context.getCatalog().changeCluster(context, enterStmt.getClusterName());
context.setDatabase("");
} catch (DdlException e) {
context.getState().setError(e.getMessage());
context.getState().setError(e.getMysqlErrorCode(), e.getMessage());
return;
}
context.getState().setOk();

View File

@ -188,10 +188,12 @@ import org.apache.doris.qe.SqlModeHelper;
keywordMap.put("exists", new Integer(SqlParserSymbols.KW_EXISTS));
keywordMap.put("explain", new Integer(SqlParserSymbols.KW_DESCRIBE));
keywordMap.put("export", new Integer(SqlParserSymbols.KW_EXPORT));
keywordMap.put("extended", new Integer(SqlParserSymbols.KW_EXTENDED));
keywordMap.put("external", new Integer(SqlParserSymbols.KW_EXTERNAL));
keywordMap.put("extract", new Integer(SqlParserSymbols.KW_EXTRACT));
keywordMap.put("false", new Integer(SqlParserSymbols.KW_FALSE));
keywordMap.put("feature", new Integer(SqlParserSymbols.KW_FEATURE));
keywordMap.put("fields", new Integer(SqlParserSymbols.KW_FIELDS));
keywordMap.put("file", new Integer(SqlParserSymbols.KW_FILE));
keywordMap.put("filter", new Integer(SqlParserSymbols.KW_FILTER));
keywordMap.put("first", new Integer(SqlParserSymbols.KW_FIRST));
@ -228,6 +230,7 @@ import org.apache.doris.qe.SqlModeHelper;
keywordMap.put("indexes", new Integer(SqlParserSymbols.KW_INDEXES));
keywordMap.put("infile", new Integer(SqlParserSymbols.KW_INFILE));
keywordMap.put("inner", new Integer(SqlParserSymbols.KW_INNER));
keywordMap.put("inner", new Integer(SqlParserSymbols.KW_INNER));
keywordMap.put("insert", new Integer(SqlParserSymbols.KW_INSERT));
keywordMap.put("int", new Integer(SqlParserSymbols.KW_INT));
keywordMap.put("integer", new Integer(SqlParserSymbols.KW_INT));
@ -258,6 +261,8 @@ import org.apache.doris.qe.SqlModeHelper;
keywordMap.put("load", new Integer(SqlParserSymbols.KW_LOAD));
keywordMap.put("local", new Integer(SqlParserSymbols.KW_LOCAL));
keywordMap.put("location", new Integer(SqlParserSymbols.KW_LOCATION));
keywordMap.put("lock", new Integer(SqlParserSymbols.KW_LOCK));
keywordMap.put("low_priority", new Integer(SqlParserSymbols.KW_LOW_PRIORITY));
keywordMap.put("map", new Integer(SqlParserSymbols.KW_MAP));
keywordMap.put("materialized", new Integer(SqlParserSymbols.KW_MATERIALIZED));
keywordMap.put("max", new Integer(SqlParserSymbols.KW_MAX));
@ -394,6 +399,7 @@ import org.apache.doris.qe.SqlModeHelper;
keywordMap.put("user", new Integer(SqlParserSymbols.KW_USER));
keywordMap.put("using", new Integer(SqlParserSymbols.KW_USING));
keywordMap.put("uninstall", new Integer(SqlParserSymbols.KW_UNINSTALL));
keywordMap.put("unlock", new Integer(SqlParserSymbols.KW_UNLOCK));
keywordMap.put("update", new Integer(SqlParserSymbols.KW_UPDATE));
keywordMap.put("value", new Integer(SqlParserSymbols.KW_VALUE));
keywordMap.put("values", new Integer(SqlParserSymbols.KW_VALUES));

View File

@ -63,7 +63,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");";
")";
String decimalTable = "CREATE TABLE `test`.`decimal_table`\n" +
"(\n" +
" `userId` varchar(255) NOT NULL COMMENT '',\n" +
@ -75,7 +75,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");";
")";
String joinTable = "CREATE TABLE `test`.`join_table`\n" +
"(\n" +
" `userId` varchar(255) NOT NULL COMMENT '',\n" +
@ -87,7 +87,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");";
")";
MetricRepo.init();
createTable(varcharTable);
createTable(decimalTable);
@ -142,7 +142,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet.getResultRows().get(0).get(1));
")", showResultSet.getResultRows().get(0).get(1));
}
@Test
@ -161,7 +161,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet1.getResultRows().get(0).get(1));
")", showResultSet1.getResultRows().get(0).get(1));
String selectFromFunction2 = "create table `test`.`select_function_2` PROPERTIES(\"replication_num\" = \"1\") as select sum(status), sum(status), sum(status), count(status), count(status) from `test`.`join_table`";
createTableAsSelect(selectFromFunction2);
@ -180,7 +180,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet2.getResultRows().get(0).get(1));
")", showResultSet2.getResultRows().get(0).get(1));
}
@Test
@ -198,7 +198,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet1.getResultRows().get(0).get(1));
")", showResultSet1.getResultRows().get(0).get(1));
String selectAlias2 = "create table `test`.`select_alias_2` PROPERTIES(\"replication_num\" = \"1\") as select userId as alias_name, username from `test`.`varchar_table`";
createTableAsSelect(selectAlias2);
ShowResultSet showResultSet2 = showCreateTable("select_alias_2");
@ -213,7 +213,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet2.getResultRows().get(0).get(1));
")", showResultSet2.getResultRows().get(0).get(1));
}
@Test
@ -234,7 +234,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet.getResultRows().get(0).get(1));
")", showResultSet.getResultRows().get(0).get(1));
}
@Test
@ -255,7 +255,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet.getResultRows().get(0).get(1));
")", showResultSet.getResultRows().get(0).get(1));
}
@Test
@ -274,7 +274,7 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet.getResultRows().get(0).get(1));
")", showResultSet.getResultRows().get(0).get(1));
}
@Test
@ -293,6 +293,6 @@ public class CreateTableAsSelectStmtTest {
"\"replication_allocation\" = \"tag.location.default: 1\",\n" +
"\"in_memory\" = \"false\",\n" +
"\"storage_format\" = \"V2\"\n" +
");", showResultSet.getResultRows().get(0).get(1));
")", showResultSet.getResultRows().get(0).get(1));
}
}

View File

@ -266,7 +266,7 @@ under the License.
<dependency>
<groupId>org.jmockit</groupId>
<artifactId>jmockit</artifactId>
<version>1.48</version>
<version>1.49</version>
<scope>test</scope>
</dependency>