add exception_status column

This commit is contained in:
bharatnc 2020-12-14 21:35:06 -08:00
parent 9004d4384d
commit c796b0c1cb

View File

@ -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
}
}
}