You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by yi...@apache.org on 2022/07/15 11:57:43 UTC

[doris] branch master updated: [feature-wip] (datev2) modify datev2 format in memory (#10873)

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

yiguolei pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new dc6fbcce14 [feature-wip] (datev2) modify datev2 format in memory (#10873)
dc6fbcce14 is described below

commit dc6fbcce14c5b10b3f5ae0f7c820415568316897
Author: Gabriel <ga...@gmail.com>
AuthorDate: Fri Jul 15 19:57:38 2022 +0800

    [feature-wip] (datev2) modify datev2 format in memory (#10873)
    
    * [feature-wip] (datev2) modify datev2 format in memory
    
    * update
---
 be/src/olap/types.h                                | 25 +++++++++-------------
 be/src/util/date_func.cpp                          |  2 +-
 be/src/vec/data_types/data_type_time_v2.cpp        |  8 ++-----
 be/src/vec/runtime/vdatetime_value.h               | 10 ++++-----
 be/test/olap/in_list_predicate_test.cpp            | 15 +++++--------
 be/test/olap/null_predicate_test.cpp               |  2 +-
 be/test/olap/storage_types_test.cpp                |  6 +++---
 be/test/vec/core/block_test.cpp                    |  2 +-
 be/test/vec/runtime/vdatetime_value_test.cpp       | 14 ++++++------
 .../org/apache/doris/analysis/DateLiteral.java     |  2 +-
 .../org/apache/doris/catalog/PrimitiveType.java    |  1 +
 .../java/org/apache/doris/catalog/ScalarType.java  |  3 +++
 .../java/org/apache/doris/alter/AlterTest.java     |  3 ---
 13 files changed, 40 insertions(+), 53 deletions(-)

diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 0f6a604aa0..2b50ad7785 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -1049,12 +1049,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATE> : public BaseFieldtypeTraits<OLAP_F
             return Status::OK();
         } else if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATEV2) {
             using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATEV2>::CppType;
-            SrcType src_value = *reinterpret_cast<const SrcType*>(src);
-            //only need part one
-            CppType year = static_cast<CppType>((src_value & 0xFFFF0000) >> 16);
-            CppType mon = static_cast<CppType>((src_value & 0xFF00) >> 8);
-            CppType mday = static_cast<CppType>(src_value & 0xFF);
-            *reinterpret_cast<CppType*>(dest) = (year << 9) + (mon << 5) + mday;
+            *reinterpret_cast<CppType*>(dest) = *reinterpret_cast<const SrcType*>(src);
             return Status::OK();
         }
 
@@ -1113,8 +1108,8 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
         char* res = strptime(scan_key.c_str(), "%Y-%m-%d", &time_tm);
 
         if (nullptr != res) {
-            uint32_t value = ((time_tm.tm_year + 1900) << 16) | ((time_tm.tm_mon + 1) << 8) |
-                             time_tm.tm_mday;
+            uint32_t value =
+                    ((time_tm.tm_year + 1900) << 9) | ((time_tm.tm_mon + 1) << 5) | time_tm.tm_mday;
             *reinterpret_cast<CppType*>(buf) = value;
         } else {
             *reinterpret_cast<CppType*>(buf) = doris::vectorized::MIN_DATE_V2;
@@ -1143,7 +1138,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
             CppType year = static_cast<CppType>((part1 / 10000L) % 10000);
             CppType mon = static_cast<CppType>((part1 / 100) % 100);
             CppType mday = static_cast<CppType>(part1 % 100);
-            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | mday;
+            *reinterpret_cast<CppType*>(dest) = (year << 9) | (mon << 5) | mday;
             return Status::OK();
         }
 
@@ -1153,7 +1148,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
             int day = static_cast<int>(value & 31);
             int mon = static_cast<int>(value >> 5 & 15);
             int year = static_cast<int>(value >> 9);
-            *reinterpret_cast<CppType*>(dest) = (year << 16) | (mon << 8) | day;
+            *reinterpret_cast<CppType*>(dest) = (year << 9) | (mon << 5) | day;
             return Status::OK();
         }
 
@@ -1167,7 +1162,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
             CppType year = static_cast<CppType>(src_value / 10000);
             CppType month = static_cast<CppType>((src_value % 10000) / 100);
             CppType day = static_cast<CppType>(src_value % 100);
-            *reinterpret_cast<CppType*>(dest) = (year << 16) | (month << 8) | day;
+            *reinterpret_cast<CppType*>(dest) = (year << 9) | (month << 5) | day;
             return Status::OK();
         }
 
@@ -1184,7 +1179,7 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATEV2>
                 if (dt.from_date_format_str(format.c_str(), format.length(), src_value.get_data(),
                                             src_value.get_size())) {
                     *reinterpret_cast<CppType*>(dest) =
-                            (dt.year() << 16) | (dt.month() << 8) | dt.day();
+                            (dt.year() << 9) | (dt.month() << 5) | dt.day();
                     return Status::OK();
                 }
             }
@@ -1256,9 +1251,9 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIME>
         } else if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATEV2) {
             using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATEV2>::CppType;
             auto value = *reinterpret_cast<const SrcType*>(src);
-            int day = static_cast<int>(value & 0xFF);
-            int mon = static_cast<int>((value & 0xFF00) >> 8);
-            int year = static_cast<int>((value & 0xFFFF0000) >> 16);
+            int day = static_cast<int>(value & 0x1F);
+            int mon = static_cast<int>((value & 0x1E0) >> 5);
+            int year = static_cast<int>((value & 0xFFFFFE00) >> 9);
             *reinterpret_cast<CppType*>(dest) = (year * 10000L + mon * 100L + day) * 1000000;
             return Status::OK();
         }
diff --git a/be/src/util/date_func.cpp b/be/src/util/date_func.cpp
index 4f7add3ad9..7dca6f1200 100644
--- a/be/src/util/date_func.cpp
+++ b/be/src/util/date_func.cpp
@@ -62,7 +62,7 @@ uint32_t timestamp_from_date_v2(const std::string& date_str) {
 
     uint32_t value = 0;
     if (nullptr != res) {
-        value = ((time_tm.tm_year + 1900) << 16) | ((time_tm.tm_mon + 1) << 8) | time_tm.tm_mday;
+        value = ((time_tm.tm_year + 1900) << 9) | ((time_tm.tm_mon + 1) << 5) | time_tm.tm_mday;
     } else {
         value = doris::vectorized::MIN_DATE_V2;
     }
diff --git a/be/src/vec/data_types/data_type_time_v2.cpp b/be/src/vec/data_types/data_type_time_v2.cpp
index ce8a7cd3eb..f9f48eed5b 100644
--- a/be/src/vec/data_types/data_type_time_v2.cpp
+++ b/be/src/vec/data_types/data_type_time_v2.cpp
@@ -30,13 +30,9 @@ std::string DataTypeDateV2::to_string(const IColumn& column, size_t row_num) con
     UInt32 int_val =
             assert_cast<const ColumnUInt32&>(*column.convert_to_full_column_if_const().get())
                     .get_data()[row_num];
+    DateV2Value val = binary_cast<UInt32, DateV2Value>(int_val);
     std::stringstream ss;
-    // Year
-    ss << (int_val & 0xFFFF0000) << '-';
-    // Month
-    ss << (int_val & 0x0000FF00) << '-';
-    // Day
-    ss << int_val % 0xFF;
+    ss << val;
     return ss.str();
 }
 
diff --git a/be/src/vec/runtime/vdatetime_value.h b/be/src/vec/runtime/vdatetime_value.h
index 9d0c58d70c..1bf0fe44dd 100644
--- a/be/src/vec/runtime/vdatetime_value.h
+++ b/be/src/vec/runtime/vdatetime_value.h
@@ -142,8 +142,8 @@ static constexpr const char* s_day_name[] = {"Monday", "Tuesday",  "Wednesday",
 static constexpr size_t MAX_DAY_NAME_LEN = max_char_length(s_day_name, std::size(s_day_name));
 static constexpr size_t MAX_MONTH_NAME_LEN = max_char_length(s_month_name, std::size(s_month_name));
 
-const uint32_t MAX_DATE_V2 = 31 | (12 << 8) | (9999 << 16);
-const uint32_t MIN_DATE_V2 = 1 | (1 << 8) | (1000 << 16);
+const uint32_t MAX_DATE_V2 = 31 | (12 << 5) | (9999 << 9);
+const uint32_t MIN_DATE_V2 = 1 | (1 << 5) | (1000 << 9);
 
 const uint32_t MAX_YEAR = 9999;
 const uint32_t MIN_YEAR = 1000;
@@ -665,9 +665,9 @@ private:
 };
 
 struct DateV2ValueType {
-    uint32_t day_ : 8;
-    uint32_t month_ : 8;
-    uint32_t year_ : 16;
+    uint32_t day_ : 5;
+    uint32_t month_ : 4;
+    uint32_t year_ : 23;
 
     DateV2ValueType(uint16_t year, uint8_t month, uint8_t day)
             : day_(day), month_(month), year_(year) {}
diff --git a/be/test/olap/in_list_predicate_test.cpp b/be/test/olap/in_list_predicate_test.cpp
index 29cb35d51c..b3f463bdd0 100644
--- a/be/test/olap/in_list_predicate_test.cpp
+++ b/be/test/olap/in_list_predicate_test.cpp
@@ -45,7 +45,7 @@ static uint32_t timestamp_from_date_v2(const char* date_string) {
     strptime(date_string, "%Y-%m-%d", &time_tm);
 
     doris::vectorized::DateV2Value value;
-    value.set_time(time_tm.tm_year, time_tm.tm_mon, time_tm.tm_mday);
+    value.set_time(time_tm.tm_year + 1900, time_tm.tm_mon + 1, time_tm.tm_mday);
     return binary_cast<doris::vectorized::DateV2Value, uint32_t>(value);
 }
 
@@ -74,15 +74,10 @@ static std::string to_date_string(uint24_t& date_value) {
 }
 
 static std::string to_date_v2_string(uint32_t& date_value) {
-    tm time_tm;
-    uint32_t value = date_value;
-    memset(&time_tm, 0, sizeof(time_tm));
-    time_tm.tm_mday = static_cast<int>(value & 0x000000FF);
-    time_tm.tm_mon = static_cast<int>((value & 0x0000FF00) >> 8);
-    time_tm.tm_year = static_cast<int>((value & 0xFFFF0000) >> 16);
-    char buf[20] = {'\0'};
-    strftime(buf, sizeof(buf), "%Y-%m-%d", &time_tm);
-    return std::string(buf);
+    auto val = binary_cast<uint32_t, vectorized::DateV2Value>(date_value);
+    std::stringstream ss;
+    ss << val;
+    return ss.str();
 }
 
 static std::string to_datetime_string(uint64_t& datetime_value) {
diff --git a/be/test/olap/null_predicate_test.cpp b/be/test/olap/null_predicate_test.cpp
index f837af19b9..fdfa168ea6 100644
--- a/be/test/olap/null_predicate_test.cpp
+++ b/be/test/olap/null_predicate_test.cpp
@@ -48,7 +48,7 @@ static uint32_t to_date_v2_timestamp(const char* date_string) {
     tm time_tm;
     strptime(date_string, "%Y-%m-%d", &time_tm);
 
-    return ((time_tm.tm_year + 1900) << 16) | ((time_tm.tm_mon + 1) << 8) | time_tm.tm_mday;
+    return ((time_tm.tm_year + 1900) << 9) | ((time_tm.tm_mon + 1) << 5) | time_tm.tm_mday;
 }
 
 }; // namespace datetime
diff --git a/be/test/olap/storage_types_test.cpp b/be/test/olap/storage_types_test.cpp
index ea1bd756d1..663a9b3d07 100644
--- a/be/test/olap/storage_types_test.cpp
+++ b/be/test/olap/storage_types_test.cpp
@@ -140,7 +140,7 @@ TEST(TypesTest, copy_and_equal) {
     common_test<OLAP_FIELD_TYPE_DATE>((1988 << 9) | (2 << 5) | 1);
     common_test<OLAP_FIELD_TYPE_DATETIME>(19880201010203L);
 
-    common_test<OLAP_FIELD_TYPE_DATEV2>((1988 << 16) | (2 << 8) | 1);
+    common_test<OLAP_FIELD_TYPE_DATEV2>((1988 << 9) | (2 << 5) | 1);
 
     Slice slice("12345abcde");
     common_test<OLAP_FIELD_TYPE_CHAR>(slice);
@@ -215,8 +215,8 @@ TEST(ArrayTypeTest, copy_and_equal) {
                               (2008 << 9) | (2 << 5) | 1};
     common_test_array<OLAP_FIELD_TYPE_DATE>(CollectionValue(date_array, 3, null_signs));
 
-    uint32_t date_v2_array[3] = {(1988 << 16) | (2 << 8) | 1, (1998 << 16) | (2 << 8) | 1,
-                                 (2008 << 16) | (2 << 8) | 1};
+    uint32_t date_v2_array[3] = {(1988 << 9) | (2 << 5) | 1, (1998 << 9) | (2 << 5) | 1,
+                                 (2008 << 9) | (2 << 5) | 1};
     common_test_array<OLAP_FIELD_TYPE_DATEV2>(CollectionValue(date_v2_array, 3, null_signs));
 
     int64_t datetime_array[3] = {19880201010203L, 19980201010203L, 20080204010203L};
diff --git a/be/test/vec/core/block_test.cpp b/be/test/vec/core/block_test.cpp
index fd87842589..ae35d95cb7 100644
--- a/be/test/vec/core/block_test.cpp
+++ b/be/test/vec/core/block_test.cpp
@@ -476,7 +476,7 @@ TEST(BlockTest, dump_data) {
     auto& date_v2_data = column_vector_date_v2->get_data();
     for (int i = 0; i < 1024; ++i) {
         vectorized::DateV2Value value;
-        value.from_date((uint32_t)((2022 << 16) | (6 << 8) | 6));
+        value.from_date((uint32_t)((2022 << 9) | (6 << 5) | 6));
         date_v2_data.push_back(*reinterpret_cast<vectorized::UInt32*>(&value));
     }
     vectorized::DataTypePtr date_v2_type(std::make_shared<vectorized::DataTypeDateV2>());
diff --git a/be/test/vec/runtime/vdatetime_value_test.cpp b/be/test/vec/runtime/vdatetime_value_test.cpp
index 727fdc2737..c445325e58 100644
--- a/be/test/vec/runtime/vdatetime_value_test.cpp
+++ b/be/test/vec/runtime/vdatetime_value_test.cpp
@@ -35,7 +35,7 @@ TEST(VDateTimeValueTest, date_v2_to_uint32_test) {
     EXPECT_TRUE(date_v2.year() == year);
     EXPECT_TRUE(date_v2.month() == month);
     EXPECT_TRUE(date_v2.day() == day);
-    EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 16) | (month << 8) | day));
+    EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
     EXPECT_TRUE(date_v2.hour() == 0);
     EXPECT_TRUE(date_v2.minute() == 0);
     EXPECT_TRUE(date_v2.second() == 0);
@@ -48,12 +48,12 @@ TEST(VDateTimeValueTest, date_v2_from_uint32_test) {
         uint8_t day = 24;
 
         DateV2Value date_v2;
-        date_v2.from_date((uint32_t)((year << 16) | (month << 8) | day));
+        date_v2.from_date((uint32_t)((year << 9) | (month << 5) | day));
 
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 16) | (month << 8) | day));
+        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
@@ -63,13 +63,13 @@ TEST(VDateTimeValueTest, date_v2_from_uint32_test) {
         uint8_t month = 5;
         uint8_t day = 24;
 
-        uint32_t ui32 = (uint32_t)((year << 16) | (month << 8) | day);
+        uint32_t ui32 = (uint32_t)((year << 9) | (month << 5) | day);
         auto date_v2 = (DateV2Value&)ui32;
 
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 16) | (month << 8) | day));
+        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
@@ -91,7 +91,7 @@ TEST(VDateTimeValueTest, date_v2_from_date_format_str_test) {
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 16) | (month << 8) | day));
+        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
@@ -107,7 +107,7 @@ TEST(VDateTimeValueTest, date_v2_from_date_format_str_test) {
         EXPECT_TRUE(date_v2.year() == year);
         EXPECT_TRUE(date_v2.month() == month);
         EXPECT_TRUE(date_v2.day() == day);
-        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 16) | (month << 8) | day));
+        EXPECT_TRUE(date_v2.to_date_uint32() == ((year << 9) | (month << 5) | day));
         EXPECT_TRUE(date_v2.hour() == 0);
         EXPECT_TRUE(date_v2.minute() == 0);
         EXPECT_TRUE(date_v2.second() == 0);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
index b3ddb4a692..52a6e58100 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DateLiteral.java
@@ -383,7 +383,7 @@ public class DateLiteral extends LiteralExpr {
         } else if (type.equals(Type.DATETIME)) {
             return (year * 10000 + month * 100 + day) * 1000000L + hour * 10000 + minute * 100 + second;
         } else if (type.equals(Type.DATEV2)) {
-            return (year << 16) | (month << 8) | day;
+            return (year << 9) | (month << 5) | day;
         } else if (type.equals(Type.DATETIMEV2)) {
             return (year << 50) | (month << 46) | (day << 41) | (hour << 36)
                 | (minute << 30) | (second << 24) | microsecond;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java
index 5cf46e53ca..240ecc037a 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/PrimitiveType.java
@@ -87,6 +87,7 @@ public enum PrimitiveType {
         builder.add(DECIMAL32);
         builder.add(DECIMAL64);
         builder.add(DECIMAL128);
+        builder.add(DATETIMEV2);
         typeWithPrecision = builder.build();
     }
 
diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
index 88231ba22d..d59adf9a57 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/ScalarType.java
@@ -721,6 +721,9 @@ public class ScalarType extends Type {
         if (isDecimalV3() && scalarType.isDecimalV3()) {
             return precision == scalarType.precision && scale == scalarType.scale;
         }
+        if (isDatetimeV2() && scalarType.isDatetimeV2()) {
+            return true;
+        }
         return false;
     }
 
diff --git a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
index a684de2404..39d3cbadea 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/alter/AlterTest.java
@@ -56,7 +56,6 @@ import com.google.common.collect.Maps;
 import org.junit.AfterClass;
 import org.junit.Assert;
 import org.junit.BeforeClass;
-import org.junit.Ignore;
 import org.junit.Test;
 
 import java.io.File;
@@ -962,8 +961,6 @@ public class AlterTest {
         alterTable(changeOrderStmt, true);
     }
 
-    // Open it when date v2 is ready
-    @Ignore
     @Test
     public void testAlterDateV2Schema() throws Exception {
         createTable("CREATE TABLE test.unique_partition_datev2\n" + "(\n" + "    k1 date,\n" + "    k2 datetime(3),\n"


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org