使用过Impala的同学都知道,impala默认对于timestamp都是当成UTC来处理的,并不会做任何的时区转换。这也就是说,当你写入一个timestamp的数据时,impala就会把它当成是UTC的时间存起来,而不是本地时间。但是Impala同时又提供了use_local_tz_for_unix_timestamp_conversions和convert_legacy_hive_parquet_utc_timestamps这两个参数来处理timestamp的时区问题。convert_legacy_hive_parquet_utc_timestamps这个参数主要是用来处理hive写parquet文件,impala读取的问题,本文暂不展开,这里主要介绍下use_local_tz_for_unix_timestamp_conversions这个参数的作用。首先,我们来看下官方的解释: The --use_local_tz_for_unix_timestamp_conversions setting affects conversions from TIMESTAMP to BIGINT, or from BIGINT to TIMESTAMP. By default, Impala treats all TIMESTAMP values as UTC, to simplify analysis of time-series data from different geographic regions. When you enable the --use_local_tz_for_unix_timestamp_conversions setting, these operations treat the input values as if they are in the local time zone of the host doing the processing. See Impala Date and Time Functions for the list of functions affected by the --use_local_tz_for_unix_timestamp_conversions setting. 简单来说,就是开启了这个参数之后(默认false,表示关闭),当SQL里面涉及到了timestamp->bigint/bigint->timestamp的转换操作时,impala会把timestamp当成是本地的时间来处理,而不是UTC时间。这个地方听起来似乎很简单,但是实际理解起来的时候非常容易出错,这里笔者将结合自己的实际测试结果来看一下use_local_tz_for_unix_timestamp_conversions这个参数究竟是如何起作用的。
测试数据准备
首先,我们将测试集群的impala的use_local_tz_for_unix_timestamp_conversions和convert_legacy_hive_parquet_utc_timestamps参数都配置为false,然后重启集群。 接着,我们使用如下SQL来创建测试表,然后插入数据:
create table timestamp_test_parquet(id int,ts timestamp,sec bigint) stored as parquet;
insert into timestamp_test_parquet values(1001,UTC_TIMESTAMP(),cast(UTC_TIMESTAMP() as bigint));
这里,为了保证一致,我们特地插入了UTC的时间,然后查询表,可以看到如下的结果:
我们执行date -d @1596003620 "+%Y-%m-%d %H:%M:%S"命令,可以得到:2020-07-29 06:20:20,现在这2个时间都是UTC的时间没问题。如果是本地时间的话,应该是2020-07-29 14:20:20(当前笔者所处的时区为东八区,即UTC+8,后面默认本地时区都是UTC+8)。
参数开启前后对比
为了验证参数里面提到的timestamp->bigint/bigint->timestamp这个转换操作,我们分别关闭/开启参数,重启集群保证参数生效,然后执行如下SQL:
select id,cast(ts as bigint),cast(sec as timestamp) from timestamp_test_parquet;
以下是两次SQL执行的结果,我们合并到了一起进行对比:
通过上述结果,我们发现,参数开启前后,两次SQL的执行结果明显是不同的。接下来,我们来根据结果进行仔细分析。
cast(ts as bigint)
当我们将timestamp转换为bigint的时候,前后两次的结果分别是:1596003620和1595974820,我们通过date命令进行转换,结果如下:
我们发现参数开启后,得到的bigint值比开启前的少了8h!这个地方要如何理解呢?结合官方的解释:参数开启之后,会将timestamp当成是本地时区的时间。所以,当我们开启参数之后,2020-07-29 06:20:20这个时间就被当成了是UTC+8的时间(默认是UTC的时间),因此在cast成bigint的时候,会把它转换成UTC的时间,所以需要减掉8h,就变成了2020-07-28 22:20:20,也就是1595974820。
cast(sec as timestamp)
当我们将bigint转换成timestamp的时候,前后2次的结果分别是:2020-07-29 06:20:20和2020-07-29 14:20:20,这里参数开启后,得到的值比开启前的多了8h!同样的道理,当参数开启之后,sec存储的是UTC的时间对应的bigint,即2020-07-29 06:20:20,所以转换成UTC+8之后,就变成了2020-07-29 14:20:20
其他生效的情况
除了上面提到的cast情况,还有一些其他的情况也会生效,例如:
select unix_timestamp(ts),FROM_UNIXTIME(sec,'yyyy-MM-dd HH:mm:ss'),TO_TIMESTAMP(sec) from timestamp_test_parquet;
结果如下:
这个结果也与上面的cast情况保持一致:当timestmap->bigint时,是会减8h;当bigint->timestmap时,是会加8h;
源码学习
我们以to_timestamp这个函数为例,看下Impala是如何根据参数进行时区转换操作的,函数信息如下:
TO_TIMESTAMP(BIGINT unixtime), TO_TIMESTAMP(STRING date, STRING pattern)
Purpose: Converts an integer or string representing a date/time value into the corresponding TIMESTAMP value.
Return type: TIMESTAMP
我们使用的是个,输入参数为bigint。首先在common/function-registry/impala_functions.py文件中找到这个udf对应的c++的函数,如下所示:
[['to_timestamp'], 'TIMESTAMP', ['BIGINT'],
'_ZN6impala18TimestampFunctions11ToTimestampEPN10impala_udf15FunctionContextERKNS1_9BigIntValE'],
其中ToTimestamp就是该udf在BE端的实际函数名,然后我们按照代码调用顺序,就能找到如下相关的代码:
//相关代码为2.12.0版本
// timestamp-functions-ir.cc
TimestampVal TimestampFunctions::ToTimestamp(FunctionContext* context,
const BigIntVal& bigint_val) {
if (bigint_val.is_null) return TimestampVal::null();
const TimestampValue& tv = TimestampValue::FromUnixTime(bigint_val.val);
TimestampVal tv_val;
tv.ToTimestampVal(&tv_val);
return tv_val;
}
// timestmap-value.h
static TimestampValue FromUnixTime(time_t unix_time) {
return TimestampValue(UnixTimeToPtime(unix_time));
}
// timestmap-value.cc
ptime TimestampValue::UnixTimeToPtime(time_t unix_time) {
if (FLAGS_use_local_tz_for_unix_timestamp_conversions) {
return UnixTimeToLocalPtime(unix_time);
} else {
return UnixTimeToUtcPtime(unix_time);
}
}
到这里就差不多可以看到了,参数开启或者关闭,Impala会调用不同的处理函数,来决定是否将输入的bigint转换为本地时间,有兴趣的读者可以自己阅读源码。3.x版本的代码整体改动比较大,可能与当前的有所不同,但是原理上应该类似。
需要注意的地方
到这里,use_local_tz_for_unix_timestamp_conversions参数,我们就已经聊的差不多了,这里有以下几点需要注意: 1. 本文只探讨use_local_tz_for_unix_timestamp_conversions这个参数的作用,不涉及convert_legacy_hive_parquet_utc_timestamps这个参数,从始至终,convert_legacy_hive_parquet_utc_timestamps都是false; 2. 笔者的测试环境为impala-2.12.0,如果是impala-3.x的版本,在执行sql之前需要设置:set timezone='Asia/Shanghai';,否则默认的时区是UTC,参数修改前后sql执行结果会一样。不同的时区请按照实际情况进行修改; 3. 这里的测试表是parquet格式,text和kudu也同样适用,这点笔者已经验证过了。其他的格式,读者有兴趣的可以自行验证,应该也是都有效果的。