dbms: moved files [#METR-2944].

This commit is contained in:
Alexey Milovidov 2015-09-24 06:50:09 +03:00
parent dc209def73
commit 75a16854db
42 changed files with 48 additions and 48 deletions

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>

View File

@ -7,7 +7,7 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/ExpressionBlockInputStream.h>

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/UnionBlockInputStream.h>

View File

@ -6,7 +6,7 @@
#include <DB/IO/WriteBufferFromFileDescriptor.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/UnionBlockInputStream.h>
@ -32,7 +32,7 @@ int main(int argc, char ** argv)
context.setPath("./");
DB::loadMetadata(context);
DB::Names column_names;
column_names.push_back("WatchID");
@ -43,7 +43,7 @@ int main(int argc, char ** argv)
for (size_t i = 0, size = streams.size(); i < size; ++i)
streams[i] = new DB::AsynchronousBlockInputStream(streams[i]);
DB::BlockInputStreamPtr stream = new DB::UnionBlockInputStream(streams, nullptr, settings.max_threads);
stream = new DB::LimitBlockInputStream(stream, 10, 0);

View File

@ -18,7 +18,7 @@
#include <DB/Storages/StorageFactory.h>
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Parsers/formatAST.h>
#include <DB/Parsers/ASTIdentifier.h>

View File

@ -9,8 +9,8 @@
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageLog.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/StorageFactory.h>
#include <DB/Interpreters/loadMetadata.h>

View File

@ -21,22 +21,22 @@
#include <DB/Common/getFQDNOrHostName.h>
#include <DB/Interpreters/loadMetadata.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemTables.h>
#include <DB/Storages/StorageSystemParts.h>
#include <DB/Storages/StorageSystemDatabases.h>
#include <DB/Storages/StorageSystemProcesses.h>
#include <DB/Storages/StorageSystemEvents.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/StorageSystemMerges.h>
#include <DB/Storages/StorageSystemSettings.h>
#include <DB/Storages/StorageSystemZooKeeper.h>
#include <DB/Storages/StorageSystemReplicas.h>
#include <DB/Storages/StorageSystemReplicationQueue.h>
#include <DB/Storages/StorageSystemDictionaries.h>
#include <DB/Storages/StorageSystemColumns.h>
#include <DB/Storages/StorageSystemFunctions.h>
#include <DB/Storages/StorageSystemClusters.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemParts.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemMerges.h>
#include <DB/Storages/System/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemZooKeeper.h>
#include <DB/Storages/System/StorageSystemReplicas.h>
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemClusters.h>
#include <DB/IO/copyData.h>
#include <DB/IO/LimitReadBuffer.h>

View File

@ -17,8 +17,8 @@
#include <DB/Storages/StorageMerge.h>
#include <DB/Storages/StorageMergeTree.h>
#include <DB/Storages/StorageDistributed.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemOne.h>
#include <DB/Storages/StorageFactory.h>
#include <DB/Storages/StorageView.h>
#include <DB/Storages/StorageMaterializedView.h>

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemClusters.h>
#include <DB/Storages/System/StorageSystemClusters.h>
#include <DB/Interpreters/Cluster.h>
#include <DB/Columns/ColumnString.h>
#include <DB/Columns/ColumnsNumber.h>

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemColumns.h>
#include <DB/Storages/System/StorageSystemColumns.h>
#include <DB/Storages/MergeTree/MergeTreeData.h>
#include <DB/Storages/StorageMergeTree.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>

View File

@ -1,7 +1,7 @@
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemDatabases.h>
#include <DB/Storages/System/StorageSystemDatabases.h>
namespace DB

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemDictionaries.h>
#include <DB/Storages/System/StorageSystemDictionaries.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeArray.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemEvents.h>
#include <DB/Storages/System/StorageSystemEvents.h>
namespace DB

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemFunctions.h>
#include <DB/Storages/System/StorageSystemFunctions.h>
#include <DB/Functions/FunctionFactory.h>
#include <DB/AggregateFunctions/AggregateFunctionFactory.h>
#include <DB/Columns/ColumnString.h>
@ -56,4 +56,4 @@ BlockInputStreams StorageSystemFunctions::read(
return BlockInputStreams{ 1, new OneBlockInputStream{{ column_name, column_is_aggregate }} };
}
}
}

View File

@ -1,4 +1,4 @@
#include <DB/Storages/StorageSystemMerges.h>
#include <DB/Storages/System/StorageSystemMerges.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/Columns/ColumnString.h>

View File

@ -5,7 +5,7 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/IProfilingBlockInputStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
namespace DB

View File

@ -4,7 +4,7 @@
#include <DB/Columns/ColumnsNumber.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemOne.h>
#include <DB/Storages/System/StorageSystemOne.h>
namespace DB

View File

@ -4,7 +4,7 @@
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataTypes/DataTypeDate.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemParts.h>
#include <DB/Storages/System/StorageSystemParts.h>
#include <DB/Storages/StorageMergeTree.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Interpreters/ProcessList.h>
#include <DB/Storages/StorageSystemProcesses.h>
#include <DB/Storages/System/StorageSystemProcesses.h>
namespace DB

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemReplicas.h>
#include <DB/Storages/System/StorageSystemReplicas.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemReplicationQueue.h>
#include <DB/Storages/System/StorageSystemReplicationQueue.h>
#include <DB/Storages/StorageReplicatedMergeTree.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -2,7 +2,7 @@
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemSettings.h>
#include <DB/Storages/System/StorageSystemSettings.h>
namespace DB

View File

@ -1,7 +1,7 @@
#include <DB/Columns/ColumnString.h>
#include <DB/DataTypes/DataTypeString.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemTables.h>
#include <DB/Storages/System/StorageSystemTables.h>
#include <DB/Common/VirtualColumnUtils.h>

View File

@ -3,7 +3,7 @@
#include <DB/DataTypes/DataTypesNumberFixed.h>
#include <DB/DataTypes/DataTypeDateTime.h>
#include <DB/DataStreams/OneBlockInputStream.h>
#include <DB/Storages/StorageSystemZooKeeper.h>
#include <DB/Storages/System/StorageSystemZooKeeper.h>
#include <DB/Parsers/ASTSelectQuery.h>
#include <DB/Parsers/ASTIdentifier.h>
#include <DB/Parsers/ASTLiteral.h>

View File

@ -3,7 +3,7 @@
#include <Poco/SharedPtr.h>
#include <DB/IO/WriteBufferFromOStream.h>
#include <DB/Storages/StorageSystemNumbers.h>
#include <DB/Storages/System/StorageSystemNumbers.h>
#include <DB/DataStreams/LimitBlockInputStream.h>
#include <DB/DataStreams/TabSeparatedRowOutputStream.h>
#include <DB/DataStreams/BlockOutputStreamFromRowOutputStream.h>
@ -31,11 +31,11 @@ int main(int argc, char ** argv)
DB::WriteBufferFromOStream out_buf(std::cout);
DB::QueryProcessingStage::Enum stage;
DB::LimitBlockInputStream input(table->read(column_names, 0, DB::Context{}, DB::Settings(), stage, 10)[0], 10, 96);
DB::RowOutputStreamPtr output_ = new DB::TabSeparatedRowOutputStream(out_buf, sample);
DB::BlockOutputStreamFromRowOutputStream output(output_);
DB::copyData(input, output);
}
catch (const DB::Exception & e)