Address PR comments

This commit is contained in:
Antonio Andelic 2022-09-01 07:56:06 +00:00
parent 7d5e13c6ba
commit c7d1dd942c
4 changed files with 63 additions and 53 deletions

33
src/Common/Base64.cpp Normal file
View File

@ -0,0 +1,33 @@
#include <Common/Base64.h>
#include <Poco/Base64Decoder.h>
#include <Poco/Base64Encoder.h>
#include <Poco/MemoryStream.h>
#include <Poco/StreamCopier.h>
#include <sstream>
namespace DB
{
std::string base64Encode(const std::string & decoded, bool url_encoding)
{
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
ostr.exceptions(std::ios::failbit);
Poco::Base64Encoder encoder(ostr, url_encoding ? Poco::BASE64_URL_ENCODING : 0);
encoder.rdbuf()->setLineLength(0);
encoder << decoded;
encoder.close();
return ostr.str();
}
std::string base64Decode(const std::string & encoded, bool url_encoding)
{
std::string decoded;
Poco::MemoryInputStream istr(encoded.data(), encoded.size());
Poco::Base64Decoder decoder(istr, url_encoding ? Poco::BASE64_URL_ENCODING : 0);
Poco::StreamCopier::copyToString(decoder, decoded);
return decoded;
}
}

10
src/Common/Base64.h Normal file
View File

@ -0,0 +1,10 @@
#pragma once
namespace DB
{
std::string base64Encode(const std::string & decoded, bool url_encoding = false);
std::string base64Decode(const std::string & encoded, bool url_encoding = false);
}

View File

@ -1,11 +1,11 @@
#include <iterator> #include <iterator>
#include <variant> #include <variant>
#include <Coordination/KeeperStorage.h>
#include <IO/Operators.h> #include <IO/Operators.h>
#include <IO/WriteHelpers.h> #include <IO/WriteHelpers.h>
#include <boost/algorithm/string.hpp> #include <boost/algorithm/string.hpp>
#include <Poco/Base64Encoder.h>
#include <Poco/SHA1Engine.h> #include <Poco/SHA1Engine.h>
#include <Common/Base64.h>
#include <Common/ZooKeeper/ZooKeeperCommon.h> #include <Common/ZooKeeper/ZooKeeperCommon.h>
#include <Common/SipHash.h> #include <Common/SipHash.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h> #include <Common/ZooKeeper/ZooKeeperConstants.h>
@ -15,8 +15,11 @@
#include <Common/logger_useful.h> #include <Common/logger_useful.h>
#include <Common/setThreadName.h> #include <Common/setThreadName.h>
#include <Common/LockMemoryExceptionInThread.h> #include <Common/LockMemoryExceptionInThread.h>
#include <Coordination/pathUtils.h> #include <Coordination/pathUtils.h>
#include <Coordination/KeeperConstants.h> #include <Coordination/KeeperConstants.h>
#include <Coordination/KeeperStorage.h>
#include <sstream> #include <sstream>
#include <iomanip> #include <iomanip>
#include <mutex> #include <mutex>
@ -36,17 +39,6 @@ namespace ErrorCodes
namespace namespace
{ {
String base64Encode(const String & decoded)
{
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
ostr.exceptions(std::ios::failbit);
Poco::Base64Encoder encoder(ostr);
encoder.rdbuf()->setLineLength(0);
encoder << decoded;
encoder.close();
return ostr.str();
}
String getSHA1(const String & userdata) String getSHA1(const String & userdata)
{ {
Poco::SHA1Engine engine; Poco::SHA1Engine engine;

View File

@ -3,6 +3,7 @@
#include <Columns/ColumnString.h> #include <Columns/ColumnString.h>
#include <Core/NamesAndTypes.h> #include <Core/NamesAndTypes.h>
#include <Core/UUID.h>
#include <DataTypes/DataTypeString.h> #include <DataTypes/DataTypeString.h>
@ -23,20 +24,17 @@
#include <Storages/StorageInMemoryMetadata.h> #include <Storages/StorageInMemoryMetadata.h>
#include <Storages/checkAndGetLiteralArgument.h> #include <Storages/checkAndGetLiteralArgument.h>
#include "Common/Exception.h" #include <Common/Base64.h>
#include "Common/ZooKeeper/IKeeper.h" #include <Common/Exception.h>
#include <Common/ZooKeeper/IKeeper.h>
#include <Common/ZooKeeper/KeeperException.h> #include <Common/ZooKeeper/KeeperException.h>
#include <Common/ZooKeeper/Types.h> #include <Common/ZooKeeper/Types.h>
#include <Common/ZooKeeper/ZooKeeper.h> #include <Common/ZooKeeper/ZooKeeper.h>
#include <Common/ZooKeeper/ZooKeeperConstants.h> #include <Common/ZooKeeper/ZooKeeperConstants.h>
#include "Core/UUID.h"
#include "base/types.h" #include <base/types.h>
#include <boost/algorithm/string/classification.hpp> #include <boost/algorithm/string/classification.hpp>
#include <Poco/Base64Decoder.h>
#include <Poco/Base64Encoder.h>
#include <Poco/MemoryStream.h>
#include <Poco/StreamCopier.h>
namespace DB namespace DB
{ {
@ -53,33 +51,11 @@ namespace ErrorCodes
namespace namespace
{ {
std::string base64Encode(const std::string & decoded)
{
std::ostringstream ostr; // STYLE_CHECK_ALLOW_STD_STRING_STREAM
ostr.exceptions(std::ios::failbit);
Poco::Base64Encoder encoder(ostr, Poco::BASE64_URL_ENCODING);
encoder.rdbuf()->setLineLength(0);
encoder << decoded;
encoder.close();
return ostr.str();
}
std::string base64Decode(const std::string & encoded)
{
std::string decoded;
Poco::MemoryInputStream istr(encoded.data(), encoded.size());
Poco::Base64Decoder decoder(istr, Poco::BASE64_URL_ENCODING);
Poco::StreamCopier::copyToString(decoder, decoded);
return decoded;
}
constexpr std::string_view default_host = "default";
std::string_view getBaseName(const std::string_view path) std::string_view getBaseName(const std::string_view path)
{ {
auto last_slash = path.find_last_of('/'); auto last_slash = path.find_last_of('/');
if (last_slash == std::string_view::npos) if (last_slash == std::string_view::npos)
throw Exception(DB::ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path); throw Exception(ErrorCodes::LOGICAL_ERROR, "Failed to get basename of path '{}'", path);
return path.substr(last_slash + 1); return path.substr(last_slash + 1);
} }
@ -98,7 +74,7 @@ public:
{ {
auto primary_key = storage.getPrimaryKey(); auto primary_key = storage.getPrimaryKey();
assert(primary_key.size() == 1); assert(primary_key.size() == 1);
primary_key_pos = getHeader().getPositionByName(storage.getPrimaryKey()[0]); primary_key_pos = getHeader().getPositionByName(primary_key[0]);
} }
std::string getName() const override { return "StorageKeeperMapSink"; } std::string getName() const override { return "StorageKeeperMapSink"; }
@ -123,7 +99,7 @@ public:
++idx; ++idx;
} }
auto key = base64Encode(wb_key.str()); auto key = base64Encode(wb_key.str(), true);
new_values[std::move(key)] = std::move(wb_value.str()); new_values[std::move(key)] = std::move(wb_value.str());
} }
} }
@ -226,7 +202,7 @@ public:
auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size); auto raw_keys = serializeKeysToRawString(it, end, key_column_type, max_block_size);
for (auto & raw_key : raw_keys) for (auto & raw_key : raw_keys)
raw_key = base64Encode(raw_key); raw_key = base64Encode(raw_key, true);
return storage.getBySerializedKeys(raw_keys, nullptr); return storage.getBySerializedKeys(raw_keys, nullptr);
} }
@ -257,7 +233,7 @@ StorageKeeperMap::StorageKeeperMap(
, keys_limit(keys_limit_) , keys_limit(keys_limit_)
, log(&Poco::Logger::get("StorageKeeperMap")) , log(&Poco::Logger::get("StorageKeeperMap"))
{ {
if (table_id.uuid == UUIDHelpers::Nil) if (!table_id.hasUUID())
throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName()); throw Exception(ErrorCodes::BAD_ARGUMENTS, "KeeperMap cannot be used with '{}' database because UUID is needed. Please use Atomic or Replicated database", table_id.getDatabaseName());
setInMemoryMetadata(metadata); setInMemoryMetadata(metadata);
@ -572,7 +548,7 @@ Chunk StorageKeeperMap::getByKeys(const ColumnsWithTypeAndName & keys, PaddedPOD
auto raw_keys = serializeKeysToRawString(keys[0]); auto raw_keys = serializeKeysToRawString(keys[0]);
if (raw_keys.size() != keys[0].column->size()) if (raw_keys.size() != keys[0].column->size())
throw DB::Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size()); throw Exception(ErrorCodes::LOGICAL_ERROR, "Assertion failed: {} != {}", raw_keys.size(), keys[0].column->size());
return getBySerializedKeys(raw_keys, &null_map); return getBySerializedKeys(raw_keys, &null_map);
} }
@ -622,7 +598,7 @@ Chunk StorageKeeperMap::getBySerializedKeys(const std::span<const std::string> k
if (code == Coordination::Error::ZOK) if (code == Coordination::Error::ZOK)
{ {
fillColumns(base64Decode(keys[i]), response.data, primary_key_pos, sample_block, columns); fillColumns(base64Decode(keys[i], true), response.data, primary_key_pos, sample_block, columns);
} }
else if (code == Coordination::Error::ZNONODE) else if (code == Coordination::Error::ZNONODE)
{ {
@ -658,11 +634,10 @@ StoragePtr create(const StorageFactory::Arguments & args)
if (engine_args.empty() || engine_args.size() > 3) if (engine_args.empty() || engine_args.size() > 3)
throw Exception( throw Exception(
ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH, ErrorCodes::NUMBER_OF_ARGUMENTS_DOESNT_MATCH,
"Storage KeeperMap requires 1-2 arguments:\n" "Storage KeeperMap requires 1-3 arguments:\n"
"root_path: path in the Keeper where the values will be stored (required)\n" "root_path: path in the Keeper where the values will be stored (required)\n"
"create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n", "create_missing_root_path: 1 if the root path should be created if it's missing, otherwise throw exception (default: 1)\n",
"keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)\n", "keys_limit: number of keys allowed to be stored, 0 is no limit (default: 0)");
default_host);
auto root_path = checkAndGetLiteralArgument<std::string>(engine_args[0], "root_path"); auto root_path = checkAndGetLiteralArgument<std::string>(engine_args[0], "root_path");