You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by zh...@apache.org on 2019/12/04 23:37:53 UTC

[incubator-doris] branch master updated: Support convert date to datetime through alter table (#2385)

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 102a845  Support convert date to datetime through alter table (#2385)
102a845 is described below

commit 102a8451311f8f6d0f87d691a13ce8d7fdfd4acd
Author: WingC <10...@qq.com>
AuthorDate: Thu Dec 5 07:37:45 2019 +0800

    Support convert date to datetime through alter table (#2385)
---
 be/src/olap/schema_change.cpp                      |  1 +
 be/src/olap/types.h                                | 13 +++++
 be/test/olap/schema_change_test.cpp                | 63 ++++++++++++++++++++++
 .../java/org/apache/doris/catalog/ColumnType.java  |  1 +
 4 files changed, 78 insertions(+)

diff --git a/be/src/olap/schema_change.cpp b/be/src/olap/schema_change.cpp
index 475a0ba..8a3f900 100644
--- a/be/src/olap/schema_change.cpp
+++ b/be/src/olap/schema_change.cpp
@@ -337,6 +337,7 @@ bool RowBlockChanger::change_row_block(
                     }
                 }
             } else if ((newtype == OLAP_FIELD_TYPE_DATE && reftype == OLAP_FIELD_TYPE_DATETIME)
+                || (newtype == OLAP_FIELD_TYPE_DATETIME && reftype == OLAP_FIELD_TYPE_DATE)
                 || (newtype == OLAP_FIELD_TYPE_DOUBLE && reftype == OLAP_FIELD_TYPE_FLOAT)) {
                 for (size_t row_index = 0, new_row_index = 0;
                         row_index < ref_block->row_block_info().row_num; ++row_index) {
diff --git a/be/src/olap/types.h b/be/src/olap/types.h
index 711c48a..08ba052 100644
--- a/be/src/olap/types.h
+++ b/be/src/olap/types.h
@@ -523,6 +523,19 @@ struct FieldTypeTraits<OLAP_FIELD_TYPE_DATETIME> : public BaseFieldtypeTraits<OL
         strftime(buf, 20, "%Y-%m-%d %H:%M:%S", &time_tm);
         return std::string(buf);
     }
+    static OLAPStatus convert_from(void* dest, const void* src, const TypeInfo* src_type, MemPool* memPool) {
+        // when convert date to datetime, automatic padding zero
+        if (src_type->type() == FieldType::OLAP_FIELD_TYPE_DATE) {
+            using SrcType = typename CppTypeTraits<OLAP_FIELD_TYPE_DATE>::CppType;
+            auto value = *reinterpret_cast<const SrcType*>(src);
+            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 * 10000L + mon * 100L + day) * 1000000;
+            return OLAPStatus::OLAP_SUCCESS;
+        }
+        return OLAPStatus::OLAP_ERR_INVALID_SCHEMA;
+    }
     static void set_to_max(void* buf) {
         // 设置为最大时间,其含义为:9999-12-31 23:59:59
         *reinterpret_cast<CppType*>(buf) = 99991231235959L;
diff --git a/be/test/olap/schema_change_test.cpp b/be/test/olap/schema_change_test.cpp
index 4efc298..871228b 100644
--- a/be/test/olap/schema_change_test.cpp
+++ b/be/test/olap/schema_change_test.cpp
@@ -377,6 +377,69 @@ TEST_F(TestColumn, ConvertDatetimeToDate) {
     ASSERT_TRUE( st == OLAP_ERR_INVALID_SCHEMA);
 }
 
+TEST_F(TestColumn, ConvertDateToDatetime) {
+    AddColumn(
+            "DateColumn",
+            "DATE",
+            "REPLACE",
+            3,
+            false,
+            true);
+    AddColumn(
+            "DateTimeColumn",
+            "DATETIME",
+            "REPLACE",
+            8,
+            false,
+            false);
+
+    TabletSchema tablet_schema;
+    InitTablet(&tablet_schema);
+    CreateColumnWriter(tablet_schema);
+
+    RowCursor write_row;
+    write_row.init(tablet_schema);
+
+    RowBlock block(&tablet_schema);
+    RowBlockInfo block_info;
+    block_info.row_num = 10000;
+    block.init(block_info);
+
+    std::vector<std::string> val_string_array;
+    std::string origin_val = "2019-12-04";
+    std::string convert_val = "2019-12-04 00:00:00";
+    val_string_array.emplace_back(origin_val);
+    val_string_array.emplace_back(convert_val);
+    OlapTuple tuple(val_string_array);
+    write_row.from_tuple(tuple);
+    block.set_row(0, write_row);
+    block.finalize(1);
+    ASSERT_EQ(_column_writer->write_batch(&block, &write_row), OLAP_SUCCESS);
+
+    ColumnDataHeaderMessage header_message;
+    ASSERT_EQ(_column_writer->finalize(&header_message), OLAP_SUCCESS);
+
+    CreateColumnReader(tablet_schema);
+    RowCursor read_row;
+    read_row.init(tablet_schema);
+    _col_vector.reset(new ColumnVector());
+    ASSERT_EQ(_column_reader->next_vector(
+            _col_vector.get(), 1, _mem_pool.get()), OLAP_SUCCESS);
+    char* data = reinterpret_cast<char*>(_col_vector->col_data());
+    read_row.set_field_content(0, data, _mem_pool.get());
+    char* src = read_row.cell_ptr(0);
+    const Field* src_field = read_row.column_schema(0);
+    read_row.convert_from(1, src, src_field->type_info(), _mem_pool.get());
+    read_row.cell_ptr(1);
+    std::string dest_string = read_row.column_schema(1)->to_string(read_row.cell_ptr(1));
+    ASSERT_TRUE(dest_string.compare(convert_val) == 0);
+
+    //test not support type
+    TypeInfo* tp = get_type_info(OLAP_FIELD_TYPE_HLL);
+    OLAPStatus st = read_row.convert_from(1, src, tp, _mem_pool.get());
+    ASSERT_TRUE( st == OLAP_ERR_INVALID_SCHEMA);
+}
+
 }
 
 int main(int argc, char** argv) {
diff --git a/fe/src/main/java/org/apache/doris/catalog/ColumnType.java b/fe/src/main/java/org/apache/doris/catalog/ColumnType.java
index 0f30e4a..ef64032 100644
--- a/fe/src/main/java/org/apache/doris/catalog/ColumnType.java
+++ b/fe/src/main/java/org/apache/doris/catalog/ColumnType.java
@@ -65,6 +65,7 @@ public abstract class ColumnType {
 
         schemaChangeMatrix[PrimitiveType.DATETIME.ordinal()][PrimitiveType.DATE.ordinal()] = true;
         schemaChangeMatrix[PrimitiveType.FLOAT.ordinal()][PrimitiveType.DOUBLE.ordinal()] = true;
+        schemaChangeMatrix[PrimitiveType.DATE.ordinal()][PrimitiveType.DATETIME.ordinal()] = true;
     }
 
     static boolean isSchemaChangeAllowed(Type lhs, Type rhs) {


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