You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/12/26 06:26:22 UTC

[GitHub] [doris] HappenLee commented on a diff in pull request #15303: [vectorized](jdbc) support array type in jdbc external table

HappenLee commented on code in PR #15303:
URL: https://github.com/apache/doris/pull/15303#discussion_r1057054824


##########
be/src/exec/table_connector.cpp:
##########
@@ -319,4 +202,148 @@ Status TableConnector::append(const std::string& table_name, vectorized::Block*
     return Status::OK();
 }
 
+Status TableConnector::convert_column_data(const vectorized::ColumnPtr& column_ptr,
+                                           const vectorized::DataTypePtr& type_ptr,
+                                           const TypeDescriptor& type, int row,
+                                           bool need_extra_convert) {
+    auto extra_convert_func = [&](const std::string_view& str, const bool& is_date) -> void {
+        if (!need_extra_convert) {
+            fmt::format_to(_insert_stmt_buffer, "'{}'", str);
+        } else {
+            if (is_date) {
+                fmt::format_to(_insert_stmt_buffer, "to_date('{}','yyyy-mm-dd')", str);
+            } else {
+                fmt::format_to(_insert_stmt_buffer, "to_date('{}','yyyy-mm-dd hh24:mi:ss')", str);
+            }
+        }
+    };
+    const vectorized::IColumn* column = column_ptr;
+    if (type_ptr->is_nullable()) {
+        if (column_ptr->is_null_at(row)) {

Review Comment:
   `assert_cast` to reduce virtual function call



##########
be/src/exec/table_connector.cpp:
##########
@@ -319,4 +202,148 @@ Status TableConnector::append(const std::string& table_name, vectorized::Block*
     return Status::OK();
 }
 
+Status TableConnector::convert_column_data(const vectorized::ColumnPtr& column_ptr,
+                                           const vectorized::DataTypePtr& type_ptr,
+                                           const TypeDescriptor& type, int row,
+                                           bool need_extra_convert) {
+    auto extra_convert_func = [&](const std::string_view& str, const bool& is_date) -> void {
+        if (!need_extra_convert) {
+            fmt::format_to(_insert_stmt_buffer, "'{}'", str);
+        } else {
+            if (is_date) {
+                fmt::format_to(_insert_stmt_buffer, "to_date('{}','yyyy-mm-dd')", str);
+            } else {
+                fmt::format_to(_insert_stmt_buffer, "to_date('{}','yyyy-mm-dd hh24:mi:ss')", str);
+            }
+        }
+    };
+    const vectorized::IColumn* column = column_ptr;
+    if (type_ptr->is_nullable()) {
+        if (column_ptr->is_null_at(row)) {
+            fmt::format_to(_insert_stmt_buffer, "{}", "NULL");
+            return Status::OK();
+        }
+        column = assert_cast<const vectorized::ColumnNullable*>(column_ptr.get())
+                         ->get_nested_column_ptr()
+                         .get();
+    } else {
+        column = column_ptr;
+    }
+    auto [item, size] = column->get_data_at(row);
+    switch (type.type) {
+    case TYPE_BOOLEAN:
+    case TYPE_TINYINT:
+        fmt::format_to(_insert_stmt_buffer, "{}", *reinterpret_cast<const int8_t*>(item));
+        break;
+    case TYPE_SMALLINT:
+        fmt::format_to(_insert_stmt_buffer, "{}", *reinterpret_cast<const int16_t*>(item));
+        break;
+    case TYPE_INT:
+        fmt::format_to(_insert_stmt_buffer, "{}", *reinterpret_cast<const int32_t*>(item));
+        break;
+    case TYPE_BIGINT:
+        fmt::format_to(_insert_stmt_buffer, "{}", *reinterpret_cast<const int64_t*>(item));
+        break;
+    case TYPE_FLOAT:
+        fmt::format_to(_insert_stmt_buffer, "{}", *reinterpret_cast<const float*>(item));
+        break;
+    case TYPE_DOUBLE:
+        fmt::format_to(_insert_stmt_buffer, "{}", *reinterpret_cast<const double*>(item));
+        break;
+    case TYPE_DATE: {
+        vectorized::VecDateTimeValue value =
+                binary_cast<int64_t, doris::vectorized::VecDateTimeValue>(*(int64_t*)item);
+
+        char buf[64];
+        char* pos = value.to_string(buf);
+        std::string_view str(buf, pos - buf - 1);
+        extra_convert_func(str, true);
+        break;
+    }
+    case TYPE_DATETIME: {
+        vectorized::VecDateTimeValue value =
+                binary_cast<int64_t, doris::vectorized::VecDateTimeValue>(*(int64_t*)item);
+
+        char buf[64];
+        char* pos = value.to_string(buf);
+        std::string_view str(buf, pos - buf - 1);
+        extra_convert_func(str, false);
+        break;
+    }
+    case TYPE_DATEV2: {
+        vectorized::DateV2Value<vectorized::DateV2ValueType> value =
+                binary_cast<uint32_t, doris::vectorized::DateV2Value<vectorized::DateV2ValueType>>(
+                        *(int32_t*)item);
+
+        char buf[64];
+        char* pos = value.to_string(buf);
+        std::string str(buf, pos - buf - 1);
+        extra_convert_func(str, true);
+        break;
+    }
+    case TYPE_DATETIMEV2: {
+        vectorized::DateV2Value<vectorized::DateTimeV2ValueType> value =
+                binary_cast<uint64_t,
+                            doris::vectorized::DateV2Value<vectorized::DateTimeV2ValueType>>(
+                        *(int64_t*)item);
+
+        char buf[64];
+        char* pos = value.to_string(buf, type.scale);
+        std::string str(buf, pos - buf - 1);
+        extra_convert_func(str, false);
+        break;
+    }
+    case TYPE_VARCHAR:
+    case TYPE_CHAR:
+    case TYPE_STRING: {
+        // here need check the ' is used, now for pg array string must be "
+        fmt::format_to(_insert_stmt_buffer, "\"{}\"", fmt::basic_string_view(item, size));
+        break;
+    }
+    case TYPE_ARRAY: {
+        auto& arr_nested = reinterpret_cast<const vectorized::ColumnArray*>(column)->get_data_ptr();
+        auto& arr_offset = reinterpret_cast<const vectorized::ColumnArray*>(column)->get_offsets();
+        auto array_type = remove_nullable(type_ptr);
+        auto nested_type =
+                reinterpret_cast<const vectorized::DataTypeArray&>(*array_type).get_nested_type();
+        //insert into doris_test.test_int values(2,'{22,33}');

Review Comment:
   why the `{}` not `[]`



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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