Merge pull request #6709 from yandex/time-zone-configuration

Fixed error with processing "timezone" in server configuration file.
This commit is contained in:
alexey-milovidov 2019-09-03 01:37:26 +03:00 committed by GitHub
commit a52874b203
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
10 changed files with 76 additions and 34 deletions

View File

@ -21,6 +21,7 @@
#include <Common/StringUtils/StringUtils.h>
#include <common/phdr_cache.h>
#include <ext/scope_guard.h>
/// Universal executable for various clickhouse applications
@ -130,8 +131,19 @@ bool isClickhouseApp(const std::string & app_suffix, std::vector<char *> & argv)
}
/// This allows to implement assert to forbid initialization of a class in static constructors.
/// Usage:
///
/// extern bool inside_main;
/// class C { C() { assert(inside_main); } };
bool inside_main = false;
int main(int argc_, char ** argv_)
{
inside_main = true;
SCOPE_EXIT({ inside_main = false; });
/// Reset new handler to default (that throws std::bad_alloc)
/// It is needed because LLVM library clobbers it.
std::set_new_handler(nullptr);

View File

@ -13,6 +13,8 @@
#include <Columns/ColumnString.h>
#include <Columns/ColumnNullable.h>
#include <Interpreters/castColumn.h>
#include <algorithm>
namespace DB
{
@ -27,34 +29,28 @@ namespace DB
extern const int CANNOT_INSERT_NULL_IN_ORDINARY_COLUMN;
extern const int THERE_IS_NO_COLUMN;
}
const std::unordered_map<arrow::Type::type, std::shared_ptr<IDataType>> arrow_type_to_internal_type = {
//{arrow::Type::DECIMAL, std::make_shared<DataTypeDecimal>()},
{arrow::Type::UINT8, std::make_shared<DataTypeUInt8>()},
{arrow::Type::INT8, std::make_shared<DataTypeInt8>()},
{arrow::Type::UINT16, std::make_shared<DataTypeUInt16>()},
{arrow::Type::INT16, std::make_shared<DataTypeInt16>()},
{arrow::Type::UINT32, std::make_shared<DataTypeUInt32>()},
{arrow::Type::INT32, std::make_shared<DataTypeInt32>()},
{arrow::Type::UINT64, std::make_shared<DataTypeUInt64>()},
{arrow::Type::INT64, std::make_shared<DataTypeInt64>()},
{arrow::Type::HALF_FLOAT, std::make_shared<DataTypeFloat32>()},
{arrow::Type::FLOAT, std::make_shared<DataTypeFloat32>()},
{arrow::Type::DOUBLE, std::make_shared<DataTypeFloat64>()},
{arrow::Type::BOOL, std::make_shared<DataTypeUInt8>()},
//{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
{arrow::Type::DATE32, std::make_shared<DataTypeDate>()},
//{arrow::Type::DATE32, std::make_shared<DataTypeDateTime>()},
{arrow::Type::DATE64, std::make_shared<DataTypeDateTime>()},
{arrow::Type::TIMESTAMP, std::make_shared<DataTypeDateTime>()},
//{arrow::Type::TIME32, std::make_shared<DataTypeDateTime>()},
static const std::initializer_list<std::pair<arrow::Type::type, const char *>> arrow_type_to_internal_type =
{
{arrow::Type::UINT8, "UInt8"},
{arrow::Type::INT8, "Int8"},
{arrow::Type::UINT16, "UInt16"},
{arrow::Type::INT16, "Int16"},
{arrow::Type::UINT32, "UInt32"},
{arrow::Type::INT32, "Int32"},
{arrow::Type::UINT64, "UInt64"},
{arrow::Type::INT64, "Int64"},
{arrow::Type::HALF_FLOAT, "Float32"},
{arrow::Type::FLOAT, "Float32"},
{arrow::Type::DOUBLE, "Float64"},
{arrow::Type::BOOL, "UInt8"},
{arrow::Type::DATE32, "Date"},
{arrow::Type::DATE64, "DateTime"},
{arrow::Type::TIMESTAMP, "DateTime"},
{arrow::Type::STRING, std::make_shared<DataTypeString>()},
{arrow::Type::BINARY, std::make_shared<DataTypeString>()},
//{arrow::Type::FIXED_SIZE_BINARY, std::make_shared<DataTypeString>()},
//{arrow::Type::UUID, std::make_shared<DataTypeString>()},
{arrow::Type::STRING, "String"},
{arrow::Type::BINARY, "String"},
// TODO: add other types that are convertable to internal ones:
// 0. ENUM?
@ -253,7 +249,7 @@ namespace DB
void ArrowColumnToCHColumn::arrowTableToCHChunk(Chunk &res, std::shared_ptr<arrow::Table> &table,
arrow::Status &read_status, const Block &header,
int &row_group_current, const Context &context, std::string format_name)
{
{
Columns columns_list;
UInt64 num_rows = 0;
@ -308,15 +304,16 @@ namespace DB
const auto decimal_type = static_cast<arrow::DecimalType *>(arrow_column->type().get());
internal_nested_type = std::make_shared<DataTypeDecimal<Decimal128>>(decimal_type->precision(),
decimal_type->scale());
} else if (arrow_type_to_internal_type.find(arrow_type) != arrow_type_to_internal_type.end())
}
else if (auto internal_type_it = std::find_if(arrow_type_to_internal_type.begin(), arrow_type_to_internal_type.end(),
[=](auto && elem) { return elem.first == arrow_type; });
internal_type_it != arrow_type_to_internal_type.end())
{
internal_nested_type = arrow_type_to_internal_type.at(arrow_type);
internal_nested_type = DataTypeFactory::instance().get(internal_type_it->second);
}
else
{
throw Exception
{
"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
throw Exception{"The type \"" + arrow_column->type()->name() + "\" of an input column \"" + arrow_column->name()
+ "\" is not supported for conversion from a " + format_name + " data format",
ErrorCodes::CANNOT_CONVERT_TYPE};
}

View File

@ -34,7 +34,8 @@ set the following environment variables:
### Running with runner script
The only requirement is fresh docker configured docker.
The only requirement is fresh configured docker and
docker pull yandex/clickhouse-integration-tests-runner
Notes:
* If you want to run integration tests without `sudo` you have to add your user to docker group `sudo usermod -aG docker $USER`. [More information](https://docs.docker.com/install/linux/linux-postinstall/) about docker configuration.

View File

@ -723,7 +723,8 @@ class ClickHouseInstance:
os.mkdir(config_d_dir)
os.mkdir(users_d_dir)
shutil.copy(p.join(HELPERS_DIR, 'common_instance_config.xml'), config_d_dir)
# The file is named with 0_ prefix to be processed before other configuration overloads.
shutil.copy(p.join(HELPERS_DIR, '0_common_instance_config.xml'), config_d_dir)
# Generate and write macros file
macros = self.macros.copy()

View File

@ -0,0 +1,4 @@
<?xml version="1.0"?>
<yandex>
<timezone>America/Los_Angeles</timezone>
</yandex>

View File

@ -0,0 +1,17 @@
import pytest
from helpers.cluster import ClickHouseCluster
cluster = ClickHouseCluster(__file__)
node = cluster.add_instance('node', main_configs=['configs/config.xml'])
@pytest.fixture(scope="module")
def start_cluster():
try:
cluster.start()
yield cluster
finally:
cluster.shutdown()
def test_check_timezone_config(start_cluster):
assert node.query("SELECT toDateTime(1111111111)") == "2005-03-17 17:58:31\n"

View File

@ -16,6 +16,7 @@
#include <memory>
#include <chrono>
#include <cstring>
#include <cassert>
#include <iostream>
#define DATE_LUT_MIN 0
@ -44,9 +45,16 @@ UInt8 getDayOfWeek(const cctz::civil_day & date)
}
__attribute__((__weak__)) extern bool inside_main;
DateLUTImpl::DateLUTImpl(const std::string & time_zone_)
: time_zone(time_zone_)
{
/// DateLUT should not be initialized in global constructors for the following reasons:
/// 1. It is too heavy.
if (&inside_main)
assert(inside_main);
size_t i = 0;
time_t start_of_day = DATE_LUT_MIN;

View File

@ -597,10 +597,12 @@ void BaseDaemon::initialize(Application & self)
/// This must be done before any usage of DateLUT. In particular, before any logging.
if (config().has("timezone"))
{
if (0 != setenv("TZ", config().getString("timezone").data(), 1))
const std::string timezone = config().getString("timezone");
if (0 != setenv("TZ", timezone.data(), 1))
throw Poco::Exception("Cannot setenv TZ variable");
tzset();
DateLUT::setDefaultTimezone(timezone);
}
std::string log_path = config().getString("logger.log", "");