mirror of
https://github.com/ClickHouse/ClickHouse.git
synced 2024-11-21 15:12:02 +00:00
add exception_status column
This commit is contained in:
parent
9004d4384d
commit
c796b0c1cb
@ -50,6 +50,37 @@ std::vector<std::pair<String, Int8>> getStatusEnumsAndValues()
|
||||
|
||||
namespace DB
|
||||
{
|
||||
std::vector<std::pair<String, Int8>> getZooKeeperErrorEnumsAndValues()
|
||||
{
|
||||
return std::vector<std::pair<String, Int8>>{
|
||||
{"ZOK", static_cast<Int8>(Coordination::Error::ZOK)},
|
||||
{"ZSYSTEMERROR", static_cast<Int8>(Coordination::Error::ZSYSTEMERROR)},
|
||||
{"ZRUNTIMEINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZRUNTIMEINCONSISTENCY)},
|
||||
{"ZDATAINCONSISTENCY", static_cast<Int8>(Coordination::Error::ZDATAINCONSISTENCY)},
|
||||
{"ZCONNECTIONLOSS", static_cast<Int8>(Coordination::Error::ZCONNECTIONLOSS)},
|
||||
{"ZMARSHALLINGERROR", static_cast<Int8>(Coordination::Error::ZMARSHALLINGERROR)},
|
||||
{"ZUNIMPLEMENTED", static_cast<Int8>(Coordination::Error::ZUNIMPLEMENTED)},
|
||||
{"ZOPERATIONTIMEOUT", static_cast<Int8>(Coordination::Error::ZOPERATIONTIMEOUT)},
|
||||
{"ZBADARGUMENTS", static_cast<Int8>(Coordination::Error::ZBADARGUMENTS)},
|
||||
{"ZINVALIDSTATE", static_cast<Int8>(Coordination::Error::ZINVALIDSTATE)},
|
||||
{"ZAPIERROR", static_cast<Int8>(Coordination::Error::ZAPIERROR)},
|
||||
{"ZNONODE", static_cast<Int8>(Coordination::Error::ZNONODE)},
|
||||
{"ZNOAUTH", static_cast<Int8>(Coordination::Error::ZNOAUTH)},
|
||||
{"ZBADVERSION", static_cast<Int8>(Coordination::Error::ZBADVERSION)},
|
||||
{"ZNOCHILDRENFOREPHEMERALS", static_cast<Int8>(Coordination::Error::ZNOCHILDRENFOREPHEMERALS)},
|
||||
{"ZNODEEXISTS", static_cast<Int8>(Coordination::Error::ZNODEEXISTS)},
|
||||
{"ZNOTEMPTY", static_cast<Int8>(Coordination::Error::ZNOTEMPTY)},
|
||||
{"ZSESSIONEXPIRED", static_cast<Int8>(Coordination::Error::ZSESSIONEXPIRED)},
|
||||
{"ZINVALIDCALLBACK", static_cast<Int8>(Coordination::Error::ZINVALIDCALLBACK)},
|
||||
{"ZINVALIDACL", static_cast<Int8>(Coordination::Error::ZINVALIDACL)},
|
||||
{"ZAUTHFAILED", static_cast<Int8>(Coordination::Error::ZAUTHFAILED)},
|
||||
{"ZCLOSING", static_cast<Int8>(Coordination::Error::ZCLOSING)},
|
||||
{"ZNOTHING", static_cast<Int8>(Coordination::Error::ZNOTHING)},
|
||||
{"ZSESSIONMOVED", static_cast<Int8>(Coordination::Error::ZSESSIONMOVED)},
|
||||
};
|
||||
}
|
||||
|
||||
|
||||
NamesAndTypesList StorageSystemDDLWorkerQueue::getNamesAndTypes()
|
||||
{
|
||||
return {
|
||||
@ -63,7 +94,7 @@ NamesAndTypesList StorageSystemDDLWorkerQueue::getNamesAndTypes()
|
||||
{"query_start_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"query_finish_time", std::make_shared<DataTypeDateTime>()},
|
||||
{"query_duration_ms", std::make_shared<DataTypeUInt64>()},
|
||||
|
||||
{"exception_code", std::make_shared<DataTypeEnum8>(getZooKeeperErrorEnumsAndValues())},
|
||||
};
|
||||
}
|
||||
|
||||
@ -139,7 +170,7 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, const C
|
||||
ErrorCodes::BAD_ARGUMENTS);
|
||||
|
||||
zkutil::ZooKeeperPtr zookeeper = context.getZooKeeper();
|
||||
|
||||
Coordination::Error zk_exception_code = Coordination::Error::ZOK;
|
||||
String ddl_zookeeper_path = config.getString("distributed_ddl.path", "/clickhouse/task_queue/ddl/");
|
||||
String ddl_query_path;
|
||||
|
||||
@ -150,9 +181,12 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, const C
|
||||
|
||||
// if there is error querying the root path of the ddl directory throw exception and stop
|
||||
zkutil::Strings queries;
|
||||
|
||||
Coordination::Error code = zookeeper->tryGetChildren(ddl_zookeeper_path, queries);
|
||||
// if there is an error here, just register the code.
|
||||
// the queries will be empty and so there will be nothing to fill the table
|
||||
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
|
||||
throw Coordination::Exception(code, ddl_zookeeper_path);
|
||||
zk_exception_code = code;
|
||||
|
||||
const auto & cluster = context.tryGetCluster(cluster_name);
|
||||
const auto & shards_info = cluster->getShardsInfo();
|
||||
@ -198,11 +232,11 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, const C
|
||||
|
||||
code = zookeeper->tryGetChildren(fs::path(ddl_query_path) / "active", active_nodes);
|
||||
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
|
||||
continue;
|
||||
zk_exception_code = code;
|
||||
|
||||
code = zookeeper->tryGetChildren(fs::path(ddl_query_path) / "finished", finished_nodes);
|
||||
if (code != Coordination::Error::ZOK && code != Coordination::Error::ZNONODE)
|
||||
continue;
|
||||
zk_exception_code = code;
|
||||
|
||||
/* status:
|
||||
* active: If the hostname:port entry is present under active path.
|
||||
@ -235,19 +269,19 @@ void StorageSystemDDLWorkerQueue::fillData(MutableColumns & res_columns, const C
|
||||
res_columns[i++]->insert(static_cast<Int8>(Status::unknown));
|
||||
}
|
||||
|
||||
// This is the original cluster_name from the query. In order to process the request, condition is WHERE should be triggered.
|
||||
// This is the original cluster_name from the query (WHERE cluster='cluster_name')
|
||||
res_columns[i++]->insert(cluster_name);
|
||||
|
||||
// values
|
||||
if (futures.empty())
|
||||
continue;
|
||||
auto res = futures[query_id].get();
|
||||
res_columns[i++]->insert(res.data);
|
||||
Coordination::GetResponse res;
|
||||
if (!futures.empty())
|
||||
res = futures[query_id].get();
|
||||
res_columns[i++]->insert(res.data); // values
|
||||
auto query_start_time = res.stat.mtime;
|
||||
|
||||
res_columns[i++]->insert(UInt64(query_start_time / 1000)); // query_start_time
|
||||
res_columns[i++]->insert(UInt64(query_finish_time / 1000)); // query_finish_time
|
||||
res_columns[i++]->insert(UInt64(query_finish_time - query_start_time)); // query_duration_ms
|
||||
res_columns[i++]->insert(static_cast<Int8>(zk_exception_code)); // query_duration_ms
|
||||
}
|
||||
}
|
||||
}
|
||||
|
Loading…
Reference in New Issue
Block a user