ClickHouse/dbms/src/Storages/StorageHDFS.cpp

249 lines
7.6 KiB
C++
Raw Normal View History

#include <Common/config.h>
#if USE_HDFS
2018-11-19 08:17:09 +00:00
#include <Storages/StorageFactory.h>
#include <Storages/StorageHDFS.h>
2018-11-19 08:17:09 +00:00
#include <Interpreters/Context.h>
#include <Interpreters/evaluateConstantExpression.h>
#include <Parsers/ASTLiteral.h>
#include <IO/ReadBufferFromHDFS.h>
#include <IO/WriteBufferFromHDFS.h>
2019-07-24 09:51:02 +00:00
#include <IO/HDFSCommon.h>
2018-11-19 08:17:09 +00:00
#include <Formats/FormatFactory.h>
#include <DataStreams/IBlockOutputStream.h>
#include <DataStreams/UnionBlockInputStream.h>
#include <DataStreams/IBlockInputStream.h>
2018-11-19 08:17:09 +00:00
#include <DataStreams/OwningBlockInputStream.h>
2019-07-24 09:51:02 +00:00
#include <Common/parseGlobs.h>
#include <Poco/URI.h>
#include <re2/re2.h>
#include <re2/stringpiece.h>
#include <hdfs/hdfs.h>
2018-11-19 08:17:09 +00:00
namespace DB
{
namespace ErrorCodes
{
extern const int NUMBER_OF_ARGUMENTS_DOESNT_MATCH;
2018-11-25 04:22:01 +00:00
extern const int NOT_IMPLEMENTED;
extern const int BAD_ARGUMENTS;
2018-11-19 08:17:09 +00:00
}
StorageHDFS::StorageHDFS(const String & uri_,
const std::string & database_name_,
2018-11-19 08:17:09 +00:00
const std::string & table_name_,
const String & format_name_,
const ColumnsDescription & columns_,
2019-08-24 21:20:20 +00:00
const ConstraintsDescription & constraints_,
Context & context_)
2019-08-24 21:20:20 +00:00
: uri(uri_)
, format_name(format_name_)
, table_name(table_name_)
, database_name(database_name_)
, context(context_)
2018-11-19 08:17:09 +00:00
{
context.getRemoteHostFilter().checkURL();
2019-08-24 21:20:20 +00:00
setColumns(columns_);
setConstraints(constraints_);
2018-11-19 08:17:09 +00:00
}
namespace
{
class HDFSBlockInputStream : public IBlockInputStream
{
public:
HDFSBlockInputStream(const String & uri,
const String & format,
const Block & sample_block,
const Context & context,
2019-02-10 16:55:12 +00:00
UInt64 max_block_size)
{
std::unique_ptr<ReadBuffer> read_buf = std::make_unique<ReadBufferFromHDFS>(uri);
auto input_stream = FormatFactory::instance().getInput(format, *read_buf, sample_block, context, max_block_size);
reader = std::make_shared<OwningBlockInputStream<ReadBuffer>>(input_stream, std::move(read_buf));
}
String getName() const override
2018-11-19 08:17:09 +00:00
{
return "HDFS";
}
Block readImpl() override
{
return reader->read();
}
Block getHeader() const override
{
return reader->getHeader();
}
void readPrefixImpl() override
{
reader->readPrefix();
}
void readSuffixImpl() override
{
reader->readSuffix();
}
private:
BlockInputStreamPtr reader;
};
class HDFSBlockOutputStream : public IBlockOutputStream
{
public:
HDFSBlockOutputStream(const String & uri,
const String & format,
const Block & sample_block_,
const Context & context)
: sample_block(sample_block_)
{
write_buf = std::make_unique<WriteBufferFromHDFS>(uri);
writer = FormatFactory::instance().getOutput(format, *write_buf, sample_block, context);
}
Block getHeader() const override
{
return sample_block;
}
void write(const Block & block) override
{
writer->write(block);
}
void writePrefix() override
{
writer->writePrefix();
}
void writeSuffix() override
{
writer->writeSuffix();
writer->flush();
write_buf->sync();
}
private:
Block sample_block;
std::unique_ptr<WriteBufferFromHDFS> write_buf;
BlockOutputStreamPtr writer;
};
2018-11-19 08:17:09 +00:00
2019-08-10 16:00:01 +00:00
/* Recursive directory listing with matched paths as a result.
* Have the same method in StorageFile.
*/
2019-08-02 15:00:12 +00:00
Strings LSWithRegexpMatching(const String & path_for_ls, const HDFSFSPtr & fs, const String & for_match)
{
2019-08-30 15:19:05 +00:00
const size_t first_glob = for_match.find_first_of("*?{");
2019-08-05 11:43:39 +00:00
2019-08-30 15:19:05 +00:00
const size_t end_of_path_without_globs = for_match.substr(0, first_glob).rfind('/');
const String suffix_with_globs = for_match.substr(end_of_path_without_globs); /// begin with '/'
const String prefix_without_globs = path_for_ls + for_match.substr(1, end_of_path_without_globs); /// ends with '/'
2019-08-05 11:43:39 +00:00
2019-08-30 15:19:05 +00:00
const size_t next_slash = suffix_with_globs.find('/', 1);
2019-08-08 14:26:02 +00:00
re2::RE2 matcher(makeRegexpPatternFromGlobs(suffix_with_globs.substr(0, next_slash)));
2019-08-02 15:00:12 +00:00
2019-08-08 14:26:02 +00:00
HDFSFileInfo ls;
ls.file_info = hdfsListDirectory(fs.get(), prefix_without_globs.data(), &ls.length);
Strings result;
for (int i = 0; i < ls.length; ++i)
{
2019-08-30 15:19:05 +00:00
const String full_path = String(ls.file_info[i].mName);
const size_t last_slash = full_path.rfind('/');
const String file_name = full_path.substr(last_slash);
const bool looking_for_directory = next_slash != std::string::npos;
2019-09-04 19:55:56 +00:00
const bool is_directory = ls.file_info[i].mKind == 'D';
2019-08-10 16:00:01 +00:00
/// Condition with type of current file_info means what kind of path is it in current iteration of ls
2019-09-04 19:55:56 +00:00
if (!is_directory && !looking_for_directory)
{
2019-08-08 14:26:02 +00:00
if (re2::RE2::FullMatch(file_name, matcher))
{
result.push_back(String(ls.file_info[i].mName));
}
}
2019-09-04 19:55:56 +00:00
else if (is_directory && looking_for_directory)
{
2019-08-08 14:26:02 +00:00
if (re2::RE2::FullMatch(file_name, matcher))
{
2019-08-08 14:26:02 +00:00
Strings result_part = LSWithRegexpMatching(full_path + "/", fs, suffix_with_globs.substr(next_slash));
2019-09-23 14:50:33 +00:00
/// Recursion depth is limited by pattern. '*' works only for depth = 1, for depth = 2 pattern path is '*/*'. So we do not need additional check.
std::move(result_part.begin(), result_part.end(), std::back_inserter(result));
}
}
}
return result;
}
2019-07-24 09:51:02 +00:00
2018-11-19 08:17:09 +00:00
}
BlockInputStreams StorageHDFS::read(
const Names & /*column_names*/,
const SelectQueryInfo & /*query_info*/,
2019-01-17 16:15:51 +00:00
const Context & context_,
2018-11-19 08:17:09 +00:00
QueryProcessingStage::Enum /*processed_stage*/,
size_t max_block_size,
2018-11-19 08:17:09 +00:00
unsigned /*num_streams*/)
{
2019-09-04 11:11:30 +00:00
const size_t begin_of_path = uri.find('/', uri.find("//") + 2);
const String path_from_uri = uri.substr(begin_of_path);
const String uri_without_path = uri.substr(0, begin_of_path);
2019-07-30 18:46:49 +00:00
2019-09-05 14:42:17 +00:00
HDFSBuilderPtr builder = createHDFSBuilder(uri_without_path + "/");
2019-07-24 09:51:02 +00:00
HDFSFSPtr fs = createHDFSFS(builder.get());
2019-08-05 11:43:39 +00:00
2019-09-04 11:11:30 +00:00
const Strings res_paths = LSWithRegexpMatching("/", fs, path_from_uri);
2019-07-24 09:51:02 +00:00
BlockInputStreams result;
for (const auto & res_path : res_paths)
2019-07-24 09:51:02 +00:00
{
result.push_back(std::make_shared<HDFSBlockInputStream>(uri_without_path + res_path, format_name, getSampleBlock(), context_,
2019-07-24 09:51:02 +00:00
max_block_size));
}
return result;
2018-11-19 08:17:09 +00:00
}
2019-08-27 23:47:30 +00:00
void StorageHDFS::rename(const String & /*new_path_to_db*/, const String & new_database_name, const String & new_table_name, TableStructureWriteLockHolder &)
{
table_name = new_table_name;
database_name = new_database_name;
}
2018-11-19 08:17:09 +00:00
BlockOutputStreamPtr StorageHDFS::write(const ASTPtr & /*query*/, const Context & /*context*/)
2018-11-19 08:17:09 +00:00
{
return std::make_shared<HDFSBlockOutputStream>(uri, format_name, getSampleBlock(), context);
2018-11-19 08:17:09 +00:00
}
void registerStorageHDFS(StorageFactory & factory)
{
factory.registerStorage("HDFS", [](const StorageFactory::Arguments & args)
{
ASTs & engine_args = args.engine_args;
if (!(engine_args.size() == 1 || engine_args.size() == 2))
throw Exception(
"Storage HDFS requires exactly 2 arguments: url and name of used format.", ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH);
engine_args[0] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[0], args.local_context);
2019-03-15 17:09:14 +00:00
String url = engine_args[0]->as<ASTLiteral &>().value.safeGet<String>();
2018-11-19 08:17:09 +00:00
engine_args[1] = evaluateConstantExpressionOrIdentifierAsLiteral(engine_args[1], args.local_context);
2019-03-15 17:09:14 +00:00
String format_name = engine_args[1]->as<ASTLiteral &>().value.safeGet<String>();
2018-11-19 08:17:09 +00:00
2019-08-24 21:20:20 +00:00
return StorageHDFS::create(url, args.database_name, args.table_name, format_name, args.columns, args.constraints, args.context);
2018-11-19 08:17:09 +00:00
});
}
}
#endif