Merge pull request #2448 from LiuYangkuan/fix-issue-2439

fix Exception in issue#2439
This commit is contained in:
alexey-milovidov 2018-06-04 23:17:44 +03:00 committed by GitHub
commit f69ecf7a9d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23

View File

@ -410,6 +410,15 @@ void StorageKafka::streamThread()
auto dependencies = context.getDependencies(database_name, table_name);
if (dependencies.size() == 0)
break;
// Check the dependencies are ready?
bool ready = true;
for (const auto & db_tab : dependencies)
{
if (!context.tryGetTable(db_tab.first, db_tab.second))
ready = false;
}
if (!ready)
break;
LOG_DEBUG(log, "Started streaming to " << dependencies.size() << " attached views");
streamToViews();