This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch branch-3.0
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/branch-3.0 by this push:
     new 93beba425a7 branch-3.0:[echcement](hive)support read hive table that 
change timestamp column to bigint. (#52954) (#53307)
93beba425a7 is described below

commit 93beba425a790558dfeab059e8fbdbd26b94b552
Author: daidai <[email protected]>
AuthorDate: Mon Jul 21 04:53:33 2025 +0800

    branch-3.0:[echcement](hive)support read hive table that change timestamp 
column to bigint. (#52954) (#53307)
    
    bp #52954
---
 be/src/vec/exec/format/column_type_convert.cpp     |  29 +++
 be/src/vec/exec/format/column_type_convert.h       |  58 ++++++
 .../exec/format/parquet/parquet_column_convert.cpp |   5 +-
 be/test/vec/exec/column_type_convert_test.cpp      | 202 +++++++++++++++++++++
 4 files changed, 291 insertions(+), 3 deletions(-)

diff --git a/be/src/vec/exec/format/column_type_convert.cpp 
b/be/src/vec/exec/format/column_type_convert.cpp
index a2c226c91d6..90efabcd847 100644
--- a/be/src/vec/exec/format/column_type_convert.cpp
+++ b/be/src/vec/exec/format/column_type_convert.cpp
@@ -61,6 +61,13 @@ namespace doris::vectorized::converter {
     M(TYPE_DATETIMEV2)           \
     M(TYPE_DATEV2)
 
+#define FOR_LOGICAL_INTEGER_TYPES(M) \
+    M(TYPE_TINYINT)                  \
+    M(TYPE_SMALLINT)                 \
+    M(TYPE_INT)                      \
+    M(TYPE_BIGINT)                   \
+    M(TYPE_LARGEINT)
+
 static bool _is_numeric_type(PrimitiveType type) {
     switch (type) {
     case TYPE_BOOLEAN:
@@ -242,6 +249,23 @@ static std::unique_ptr<ColumnTypeConverter> 
_numeric_to_decimal_converter(
     }
 }
 
+static std::unique_ptr<ColumnTypeConverter> _datetime_to_numeric_converter(
+        const TypeDescriptor& src_type, const DataTypePtr& dst_type) {
+    PrimitiveType dst_primitive_type =
+            remove_nullable(dst_type)->get_type_as_type_descriptor().type;
+    switch (dst_primitive_type) {
+#define DISPATCH(DST_TYPE)                                               \
+    case DST_TYPE: {                                                     \
+        return std::make_unique<DateTimeToNumericConverter<DST_TYPE>>(); \
+    }
+        FOR_LOGICAL_INTEGER_TYPES(DISPATCH)
+#undef DISPATCH
+    default: {
+        return std::make_unique<UnsupportedConverter>(src_type, dst_type);
+    }
+    };
+}
+
 static std::unique_ptr<ColumnTypeConverter> _decimal_to_numeric_converter(
         const TypeDescriptor& src_type, const DataTypePtr& dst_type) {
     PrimitiveType src_primitive_type = src_type.type;
@@ -316,6 +340,11 @@ std::unique_ptr<ColumnTypeConverter> 
ColumnTypeConverter::get_converter(
         return std::make_unique<TimeV2Converter<TYPE_DATETIMEV2, 
TYPE_DATEV2>>();
     }
 
+    // datetime to bigint (ms)
+    if (src_primitive_type == TYPE_DATETIMEV2 && 
_is_numeric_type(dst_primitive_type)) {
+        return _datetime_to_numeric_converter(src_type, dst_type);
+    }
+
     // numeric to decimal
     if (_is_numeric_type(src_primitive_type) && 
_is_decimal_type(dst_primitive_type)) {
         return _numeric_to_decimal_converter(src_type, dst_type);
diff --git a/be/src/vec/exec/format/column_type_convert.h 
b/be/src/vec/exec/format/column_type_convert.h
index d4a8186549a..0825a5f0186 100644
--- a/be/src/vec/exec/format/column_type_convert.h
+++ b/be/src/vec/exec/format/column_type_convert.h
@@ -17,6 +17,8 @@
 
 #pragma once
 
+#include <cctz/time_zone.h>
+
 #include "gutil/strings/numbers.h"
 #include "vec/columns/column_string.h"
 #include "vec/core/types.h"
@@ -412,6 +414,62 @@ public:
     }
 };
 
+template <PrimitiveType DstPrimitiveType>
+class DateTimeToNumericConverter : public ColumnTypeConverter {
+public:
+    Status convert(ColumnPtr& src_col, MutableColumnPtr& dst_col) override {
+        using SrcColumnType = typename 
PrimitiveTypeTraits<TYPE_DATETIMEV2>::ColumnType;
+        using DstColumnType = typename 
PrimitiveTypeTraits<DstPrimitiveType>::ColumnType;
+        using SrcCppType = typename 
PrimitiveTypeTraits<TYPE_DATETIMEV2>::CppType;
+        using DstCppType = typename 
PrimitiveTypeTraits<DstPrimitiveType>::CppType;
+
+        ColumnPtr from_col = remove_nullable(src_col);
+        MutableColumnPtr to_col = 
remove_nullable(dst_col->get_ptr())->assume_mutable();
+
+        NullMap* null_map = nullptr;
+        if (dst_col->is_nullable()) {
+            null_map = 
&reinterpret_cast<vectorized::ColumnNullable*>(dst_col.get())
+                                ->get_null_map_data();
+        }
+
+        size_t rows = from_col->size();
+        auto& src_data = static_cast<const 
SrcColumnType*>(from_col.get())->get_data();
+        size_t start_idx = to_col->size();
+        to_col->resize(start_idx + rows);
+        auto& data = static_cast<DstColumnType&>(*to_col.get()).get_data();
+
+        for (int i = 0; i < rows; ++i) {
+            const SrcCppType& src_value = src_data[i];
+            auto& dst_value = reinterpret_cast<DstCppType&>(data[start_idx + 
i]);
+
+            int64_t ts_s = 0;
+            if (!src_value.unix_timestamp(&ts_s, cctz::utc_time_zone())) {
+                if (null_map == nullptr) {
+                    return Status::InternalError("Failed to cast value '{}' to 
{} column",
+                                                 src_data[i], 
dst_col->get_name());
+                } else {
+                    (*null_map)[start_idx + i] = 1;
+                }
+            }
+            auto micro = src_value.microsecond();
+            int64_t ts_ms = ts_s * 1000 + micro / 1000;
+            if constexpr (DstPrimitiveType != TYPE_LARGEINT && 
DstPrimitiveType != TYPE_BIGINT) {
+                if ((Int64)std::numeric_limits<DstCppType>::min() > ts_ms ||
+                    ts_ms > (Int64)std::numeric_limits<DstCppType>::max()) {
+                    if (null_map == nullptr) {
+                        return Status::InternalError("Failed to cast value 
'{}' to {} column",
+                                                     src_data[i], 
dst_col->get_name());
+                    } else {
+                        (*null_map)[start_idx + i] = 1;
+                    }
+                }
+            }
+            dst_value = static_cast<DstCppType>(ts_ms);
+        }
+        return Status::OK();
+    }
+};
+
 // only support date & datetime v2
 template <PrimitiveType SrcPrimitiveType, PrimitiveType DstPrimitiveType>
 class TimeV2Converter : public ColumnTypeConverter {
diff --git a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp 
b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp
index 0a5ef2913dd..a80a26390db 100644
--- a/be/src/vec/exec/format/parquet/parquet_column_convert.cpp
+++ b/be/src/vec/exec/format/parquet/parquet_column_convert.cpp
@@ -286,9 +286,8 @@ std::unique_ptr<PhysicalToLogicalConverter> 
PhysicalToLogicalConverter::get_conv
             convert_params->reset_time_scale_if_missing(9);
             physical_converter.reset(new Int96toTimestamp());
         } else if (src_physical_type == tparquet::Type::INT64) {
-            convert_params->reset_time_scale_if_missing(
-                    remove_nullable(dst_logical_type)->get_scale());
-            physical_converter.reset(new Int64ToTimestamp());
+            
convert_params->reset_time_scale_if_missing(src_logical_type.scale);
+            physical_converter = std::make_unique<Int64ToTimestamp>();
         } else {
             physical_converter.reset(new 
UnsupportedConverter(src_physical_type, src_logical_type));
         }
diff --git a/be/test/vec/exec/column_type_convert_test.cpp 
b/be/test/vec/exec/column_type_convert_test.cpp
new file mode 100644
index 00000000000..f883b00ce29
--- /dev/null
+++ b/be/test/vec/exec/column_type_convert_test.cpp
@@ -0,0 +1,202 @@
+// Licensed to the Apache Software Foundation (ASF) under one
+// or more contributor license agreements.  See the NOTICE file
+// distributed with this work for additional information
+// regarding copyright ownership.  The ASF licenses this file
+// to you under the Apache License, Version 2.0 (the
+// "License"); you may not use this file except in compliance
+// with the License.  You may obtain a copy of the License at
+//
+//   http://www.apache.org/licenses/LICENSE-2.0
+//
+// Unless required by applicable law or agreed to in writing,
+// software distributed under the License is distributed on an
+// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+// KIND, either express or implied.  See the License for the
+// specific language governing permissions and limitations
+// under the License.
+
+#include "vec/exec/format/column_type_convert.h"
+
+#include <gtest/gtest.h>
+
+#include <limits>
+
+#include "vec/columns/column_decimal.h"
+#include "vec/columns/column_nullable.h"
+#include "vec/columns/column_string.h"
+#include "vec/core/types.h"
+#include "vec/data_types/data_type_date.h"
+#include "vec/data_types/data_type_date_time.h"
+#include "vec/data_types/data_type_decimal.h"
+#include "vec/data_types/data_type_factory.hpp"
+#include "vec/data_types/data_type_nullable.h"
+#include "vec/data_types/data_type_number.h"
+#include "vec/data_types/data_type_string.h"
+
+namespace doris::vectorized {
+
+class ColumnTypeConverterTest : public testing::Test {
+public:
+    ColumnTypeConverterTest() = default;
+    virtual ~ColumnTypeConverterTest() = default;
+};
+
+TEST_F(ColumnTypeConverterTest, TestDateTimeV2ToNumericConversions) {
+    using namespace doris::vectorized;
+    auto make_datetimev2_col =
+            [](const std::vector<std::tuple<int, int, int, int, int, int, 
int>>& datetimes) {
+                auto col = ColumnDateTimeV2::create();
+                for (const auto& [y, m, d, h, min, s, micro] : datetimes) {
+                    DateV2Value<DateTimeV2ValueType> v;
+                    v.unchecked_set_time(y, m, d, h, min, s, micro);
+                    
col->get_data().push_back(*reinterpret_cast<vectorized::UInt64*>(&v));
+                }
+                return col;
+            };
+
+    auto parse_datetimev2_str = [](const std::string& datetime_str) {
+        UInt64 x = 0;
+        ReadBuffer buf((char*)datetime_str.data(), datetime_str.size());
+        bool ok = read_datetime_v2_text_impl(x, buf, 6);
+        CHECK(ok) << "parse_datetimev2_str failed for: " << datetime_str;
+        return x;
+    };
+
+    // 1. DATETIMEV2 -> BIGINT
+    {
+        TypeDescriptor src_type(TYPE_DATETIMEV2);
+        auto dst_type = std::make_shared<DataTypeInt64>();
+        auto converter = 
converter::ColumnTypeConverter::get_converter(src_type, dst_type);
+
+        ASSERT_TRUE(converter->support());
+
+        // 2024-01-01 00:00:00.123456
+        auto src_col = make_datetimev2_col({{2024, 1, 1, 0, 0, 0, 123456}});
+        auto dst_col = dst_type->create_column();
+        auto mutable_dst = dst_col->assume_mutable();
+
+        Status st = converter->convert(reinterpret_cast<ColumnPtr&>(src_col), 
mutable_dst);
+        ASSERT_TRUE(st.ok());
+
+        auto& dst_data = static_cast<ColumnInt64&>(*mutable_dst).get_data();
+        ASSERT_EQ(1, dst_data.size());
+        EXPECT_EQ(1704067200123, dst_data[0]);
+    }
+
+    // 2. DATETIMEV2 -> INT
+    {
+        TypeDescriptor src_type(TYPE_DATETIMEV2);
+        auto dst_type = std::make_shared<DataTypeInt32>();
+        auto nullable_dst_type = std::make_shared<DataTypeNullable>(dst_type);
+        auto converter = 
converter::ColumnTypeConverter::get_converter(src_type, nullable_dst_type);
+
+        ASSERT_TRUE(converter->support());
+
+        // 1970-01-01 00:00:00.000000
+        // 3000-01-01 00:00:00.000000
+        auto src_col = make_datetimev2_col({{1970, 1, 1, 0, 0, 0, 0}, {3000, 
1, 1, 0, 0, 0, 0}});
+        auto dst_col = nullable_dst_type->create_column();
+        auto mutable_dst = dst_col->assume_mutable();
+        auto& nullable_col = static_cast<ColumnNullable&>(*mutable_dst);
+        auto& null_map = nullable_col.get_null_map_data();
+        null_map.resize_fill(src_col->size(), 0);
+
+        Status st = converter->convert(reinterpret_cast<ColumnPtr&>(src_col), 
mutable_dst);
+        ASSERT_TRUE(st.ok());
+        auto& nested_col = 
static_cast<ColumnInt32&>(nullable_col.get_nested_column());
+        auto& dst_data = nested_col.get_data();
+
+        ASSERT_EQ(2, nested_col.size());
+        EXPECT_EQ(0, null_map[0]);
+        ASSERT_EQ(0, dst_data[0]);
+        EXPECT_EQ(1, null_map[1]);
+    }
+
+    // 3. DATETIMEV2 -> INT, non-nullable
+    {
+        TypeDescriptor src_type(TYPE_DATETIMEV2);
+        auto dst_type = std::make_shared<DataTypeInt32>();
+        auto converter = 
converter::ColumnTypeConverter::get_converter(src_type, dst_type);
+
+        ASSERT_TRUE(converter->support());
+
+        // 3000-01-01 00:00:00.000000(会溢出int32)
+        auto src_col = make_datetimev2_col({{3000, 1, 1, 0, 0, 0, 0}});
+        auto dst_col = dst_type->create_column();
+        auto mutable_dst = dst_col->assume_mutable();
+
+        Status st = converter->convert(reinterpret_cast<ColumnPtr&>(src_col), 
mutable_dst);
+        ASSERT_FALSE(st.ok());
+    }
+
+    {
+        TypeDescriptor src_type(TYPE_DATETIMEV2);
+        auto dst_type = std::make_shared<DataTypeInt64>();
+        auto nullable_dst_type = std::make_shared<DataTypeNullable>(dst_type);
+        auto converter = 
converter::ColumnTypeConverter::get_converter(src_type, nullable_dst_type);
+
+        ASSERT_TRUE(converter->support());
+
+        auto src_col = ColumnDateTimeV2::create();
+        src_col->get_data().push_back(parse_datetimev2_str("2024-01-01 
12:34:56.123456"));
+        src_col->get_data().push_back(parse_datetimev2_str("1970-01-01 
00:00:00.000000"));
+        src_col->get_data().push_back(parse_datetimev2_str("3000-01-01 
00:00:00.000000"));
+        src_col->get_data().push_back(parse_datetimev2_str("1900-01-01 
00:00:00.000000"));
+        src_col->get_data().push_back(parse_datetimev2_str("1999-12-31 
23:59:59.999999"));
+        src_col->get_data().push_back(parse_datetimev2_str("2000-01-01 
00:00:00.000000"));
+        src_col->get_data().push_back(parse_datetimev2_str("2025-07-08 
16:00:00.123456"));
+        src_col->get_data().push_back(parse_datetimev2_str("2100-01-01 
00:00:00.000000"));
+        src_col->get_data().push_back(parse_datetimev2_str("9999-12-31 
23:59:59.999999"));
+        src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 
12:00:00.000001"));
+        src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 
13:00:00.000002"));
+        src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 
14:00:00.000004"));
+        src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 
12:00:00"));
+        src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 
13:00:00"));
+        src_col->get_data().push_back(parse_datetimev2_str("2022-05-01 
14:00:00"));
+
+        auto dst_col = nullable_dst_type->create_column();
+        auto mutable_dst = dst_col->assume_mutable();
+        auto& nullable_col = static_cast<ColumnNullable&>(*mutable_dst);
+        auto& null_map = nullable_col.get_null_map_data();
+        null_map.resize_fill(src_col->size(), 0);
+
+        Status st = converter->convert(reinterpret_cast<ColumnPtr&>(src_col), 
mutable_dst);
+        ASSERT_TRUE(st.ok());
+
+        ASSERT_EQ(15, null_map.size());
+        EXPECT_EQ(0, null_map[0]);
+        EXPECT_EQ(0, null_map[1]);
+        EXPECT_EQ(0, null_map[2]);
+        EXPECT_EQ(0, null_map[3]);
+        EXPECT_EQ(0, null_map[4]);
+        EXPECT_EQ(0, null_map[5]);
+        EXPECT_EQ(0, null_map[6]);
+        EXPECT_EQ(0, null_map[7]);
+        EXPECT_EQ(0, null_map[8]);
+        EXPECT_EQ(0, null_map[9]);
+        EXPECT_EQ(0, null_map[10]);
+        EXPECT_EQ(0, null_map[11]);
+        EXPECT_EQ(0, null_map[12]);
+        EXPECT_EQ(0, null_map[13]);
+        EXPECT_EQ(0, null_map[14]);
+
+        auto& dst_data = 
static_cast<ColumnInt64&>(nullable_col.get_nested_column()).get_data();
+        ASSERT_EQ(15, dst_data.size());
+        EXPECT_EQ(1704112496123L, dst_data[0]);
+        EXPECT_EQ(0L, dst_data[1]);
+        EXPECT_EQ(32503680000000L, dst_data[2]);
+        EXPECT_EQ(-2208988800000L, dst_data[3]);
+        EXPECT_EQ(946684799999L, dst_data[4]);
+        EXPECT_EQ(946684800000L, dst_data[5]);
+        EXPECT_EQ(1751990400123, dst_data[6]);
+        EXPECT_EQ(4102444800000L, dst_data[7]);
+        EXPECT_EQ(253402300799999, dst_data[8]);
+        EXPECT_EQ(1651406400000, dst_data[9]);
+        EXPECT_EQ(1651410000000, dst_data[10]);
+        EXPECT_EQ(1651413600000, dst_data[11]);
+        EXPECT_EQ(1651406400000, dst_data[12]);
+        EXPECT_EQ(1651410000000, dst_data[13]);
+        EXPECT_EQ(1651413600000, dst_data[14]);
+    }
+}
+} // namespace doris::vectorized
\ No newline at end of file


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to