lidavidm commented on code in PR #40939: URL: https://github.com/apache/arrow/pull/40939#discussion_r2099321922
########## cpp/src/arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.h: ########## @@ -0,0 +1,32 @@ +// 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. + +#pragma once + +#include <arrow/type.h> + +namespace driver { +namespace flight_sql { + +void ReportSystemFunction(const std::string& function, uint32_t& current_sys_functions, + uint32_t& current_convert_functions); +void ReportNumericFunction(const std::string& function, uint32_t& current_functions); Review Comment: Can the purpose of these functions be documented? Even looking at the implementation it's not very clear ########## cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/diagnostics.h: ########## @@ -0,0 +1,109 @@ +// 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. + +#pragma once + +#include <memory> +#include <string> +#include <vector> + +#include <arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/exceptions.h> +#include <arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/types.h> + +namespace driver { +namespace odbcabstraction { +class Diagnostics { + public: + struct DiagnosticsRecord { + std::string msg_text_; + std::string sql_state_; + int32_t native_error_; + }; + + private: + std::vector<const DiagnosticsRecord*> error_records_; + std::vector<const DiagnosticsRecord*> warning_records_; + std::vector<std::unique_ptr<DiagnosticsRecord>> owned_records_; + std::string vendor_; + std::string data_source_component_; + OdbcVersion version_; + + public: + Diagnostics(std::string vendor, std::string data_source_component, OdbcVersion version); + void AddError(const DriverException& exception); + void AddWarning(std::string message, std::string sql_state, int32_t native_error); + + /// \brief Add a pre-existing truncation warning. + inline void AddTruncationWarning() { + static const std::unique_ptr<DiagnosticsRecord> TRUNCATION_WARNING( + new DiagnosticsRecord{"String or binary data, right-truncated.", "01004", + ODBCErrorCodes_TRUNCATION_WARNING}); + warning_records_.push_back(TRUNCATION_WARNING.get()); Review Comment: I think this one doesn't need to be a unique_ptr? It shouldn't ever get moved. ########## cpp/src/arrow/flight/sql/odbc/flight_sql/flight_sql_statement.cc: ########## @@ -0,0 +1,298 @@ +// 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 "arrow/flight/sql/odbc/flight_sql/flight_sql_statement.h" +#include <sql.h> +#include <sqlext.h> +#include "arrow/flight/sql/odbc/flight_sql/flight_sql_result_set.h" +#include "arrow/flight/sql/odbc/flight_sql/flight_sql_result_set_metadata.h" +#include "arrow/flight/sql/odbc/flight_sql/flight_sql_statement_get_columns.h" +#include "arrow/flight/sql/odbc/flight_sql/flight_sql_statement_get_tables.h" +#include "arrow/flight/sql/odbc/flight_sql/flight_sql_statement_get_type_info.h" +#include "arrow/flight/sql/odbc/flight_sql/record_batch_transformer.h" +#include "arrow/flight/sql/odbc/flight_sql/utils.h" +#include "arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h" +#include "arrow/io/memory.h" + +#include <boost/optional.hpp> +#include <utility> +#include "arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/exceptions.h" + +namespace driver { +namespace flight_sql { + +using arrow::Result; +using arrow::Status; +using arrow::flight::FlightCallOptions; +using arrow::flight::FlightClientOptions; +using arrow::flight::FlightInfo; +using arrow::flight::Location; +using arrow::flight::TimeoutDuration; +using arrow::flight::sql::FlightSqlClient; +using arrow::flight::sql::PreparedStatement; +using driver::odbcabstraction::DriverException; +using driver::odbcabstraction::ResultSet; +using driver::odbcabstraction::ResultSetMetadata; +using driver::odbcabstraction::Statement; + +namespace { + +void ClosePreparedStatementIfAny( + std::shared_ptr<arrow::flight::sql::PreparedStatement>& prepared_statement) { + if (prepared_statement != nullptr) { + ThrowIfNotOK(prepared_statement->Close()); + prepared_statement.reset(); + } +} + +} // namespace + +FlightSqlStatement::FlightSqlStatement( + const odbcabstraction::Diagnostics& diagnostics, FlightSqlClient& sql_client, + FlightCallOptions call_options, + const odbcabstraction::MetadataSettings& metadata_settings) + : diagnostics_("Apache Arrow", diagnostics.GetDataSourceComponent(), + diagnostics.GetOdbcVersion()), + sql_client_(sql_client), + call_options_(std::move(call_options)), + metadata_settings_(metadata_settings) { + attribute_[METADATA_ID] = static_cast<size_t>(SQL_FALSE); + attribute_[MAX_LENGTH] = static_cast<size_t>(0); + attribute_[NOSCAN] = static_cast<size_t>(SQL_NOSCAN_OFF); + attribute_[QUERY_TIMEOUT] = static_cast<size_t>(0); + call_options_.timeout = TimeoutDuration{-1}; +} + +bool FlightSqlStatement::SetAttribute(StatementAttributeId attribute, + const Attribute& value) { + switch (attribute) { + case METADATA_ID: + return CheckIfSetToOnlyValidValue(value, static_cast<size_t>(SQL_FALSE)); + case NOSCAN: + return CheckIfSetToOnlyValidValue(value, static_cast<size_t>(SQL_NOSCAN_OFF)); + case MAX_LENGTH: + return CheckIfSetToOnlyValidValue(value, static_cast<size_t>(0)); + case QUERY_TIMEOUT: + if (boost::get<size_t>(value) > 0) { + call_options_.timeout = + TimeoutDuration{static_cast<double>(boost::get<size_t>(value))}; + } else { + call_options_.timeout = TimeoutDuration{-1}; + // Intentional fall-through. + } + default: + attribute_[attribute] = value; + return true; + } +} + +boost::optional<Statement::Attribute> FlightSqlStatement::GetAttribute( + StatementAttributeId attribute) { + const auto& it = attribute_.find(attribute); + return boost::make_optional(it != attribute_.end(), it->second); +} + +boost::optional<std::shared_ptr<ResultSetMetadata>> FlightSqlStatement::Prepare( + const std::string& query) { + ClosePreparedStatementIfAny(prepared_statement_); + + Result<std::shared_ptr<PreparedStatement>> result = + sql_client_.Prepare(call_options_, query); + ThrowIfNotOK(result.status()); + + prepared_statement_ = *result; + + const auto& result_set_metadata = std::make_shared<FlightSqlResultSetMetadata>( + prepared_statement_->dataset_schema(), metadata_settings_); + return boost::optional<std::shared_ptr<ResultSetMetadata>>(result_set_metadata); +} + +bool FlightSqlStatement::ExecutePrepared() { + assert(prepared_statement_.get() != nullptr); Review Comment: use Arrow DCHECK macros? ########## cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/blocking_queue.h: ########## @@ -0,0 +1,131 @@ +// 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. + +#pragma once + +#include <atomic> +#include <boost/optional.hpp> +#include <condition_variable> +#include <mutex> +#include <thread> +#include <vector> + +namespace driver { +namespace odbcabstraction { + +template <typename T> +class BlockingQueue { + size_t capacity_; + std::vector<T> buffer_; + size_t buffer_size_{0}; + size_t left_{0}; // index where variables are put inside of buffer (produced) + size_t right_{0}; // index where variables are removed from buffer (consumed) + + std::mutex mtx_; + std::condition_variable not_empty_; + std::condition_variable not_full_; + + std::vector<std::thread> threads_; + std::atomic<size_t> active_threads_{0}; + std::atomic<bool> closed_{false}; + + public: + typedef std::function<boost::optional<T>(void)> Supplier; + + explicit BlockingQueue(size_t capacity) : capacity_(capacity), buffer_(capacity) {} + + void AddProducer(Supplier supplier) { + active_threads_++; + threads_.emplace_back([=] { + while (!closed_) { + // Block while queue is full + std::unique_lock<std::mutex> unique_lock(mtx_); + if (!WaitUntilCanPushOrClosed(unique_lock)) break; + unique_lock.unlock(); Review Comment: Shouldn't we check if `closed_` and break? ########## cpp/src/arrow/flight/sql/odbc/flight_sql/record_batch_transformer.cc: ########## @@ -0,0 +1,149 @@ +// 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 "arrow/flight/sql/odbc/flight_sql/record_batch_transformer.h" +#include "arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h" + +#include <iostream> +#include <utility> +#include "arrow/array/util.h" +#include "arrow/builder.h" +#include "arrow/flight/sql/odbc/flight_sql/utils.h" + +#include "arrow/array/array_base.h" + +namespace driver { +namespace flight_sql { + +using arrow::ArrayBuilder; +using arrow::MemoryPool; +using arrow::Result; + +namespace { +Result<std::shared_ptr<Array>> MakeEmptyArray(std::shared_ptr<DataType> type, Review Comment: Use MakeArrayOfNull ########## cpp/src/arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.cc: ########## @@ -0,0 +1,140 @@ +// 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 "arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.h" + +#include "arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h" + +#include <sqlext.h> +#include <string> +#include <unordered_map> + +namespace driver { +namespace flight_sql { + +// The list of functions that can be converted from string to ODBC bitmasks is +// based on Calcite's SqlJdbcFunctionCall class. + +namespace { +static const std::unordered_map<std::string, uint32_t> numeric_functions = { + {"ABS", SQL_FN_NUM_ABS}, {"ACOS", SQL_FN_NUM_ACOS}, + {"ASIN", SQL_FN_NUM_ASIN}, {"ATAN", SQL_FN_NUM_ATAN}, + {"ATAN2", SQL_FN_NUM_ATAN2}, {"CEILING", SQL_FN_NUM_CEILING}, + {"COS", SQL_FN_NUM_ACOS}, {"COT", SQL_FN_NUM_COT}, + {"DEGREES", SQL_FN_NUM_DEGREES}, {"EXP", SQL_FN_NUM_EXP}, + {"FLOOR", SQL_FN_NUM_FLOOR}, {"LOG", SQL_FN_NUM_LOG}, + {"LOG10", SQL_FN_NUM_LOG10}, {"MOD", SQL_FN_NUM_MOD}, + {"PI", SQL_FN_NUM_PI}, {"POWER", SQL_FN_NUM_POWER}, + {"RADIANS", SQL_FN_NUM_RADIANS}, {"RAND", SQL_FN_NUM_RAND}, + {"ROUND", SQL_FN_NUM_ROUND}, {"SIGN", SQL_FN_NUM_SIGN}, + {"SIN", SQL_FN_NUM_SIN}, {"SQRT", SQL_FN_NUM_SQRT}, + {"TAN", SQL_FN_NUM_TAN}, {"TRUNCATE", SQL_FN_NUM_TRUNCATE}}; + +static const std::unordered_map<std::string, uint32_t> system_functions = { + {"DATABASE", SQL_FN_SYS_DBNAME}, + {"IFNULL", SQL_FN_SYS_IFNULL}, + {"USER", SQL_FN_SYS_USERNAME}}; + +static const std::unordered_map<std::string, uint32_t> datetime_functions = { + {"CURDATE", SQL_FN_TD_CURDATE}, + {"CURTIME", SQL_FN_TD_CURTIME}, + {"DAYNAME", SQL_FN_TD_DAYNAME}, + {"DAYOFMONTH", SQL_FN_TD_DAYOFMONTH}, + {"DAYOFWEEK", SQL_FN_TD_DAYOFWEEK}, + {"DAYOFYEAR", SQL_FN_TD_DAYOFYEAR}, + {"HOUR", SQL_FN_TD_HOUR}, + {"MINUTE", SQL_FN_TD_MINUTE}, + {"MONTH", SQL_FN_TD_MONTH}, + {"MONTHNAME", SQL_FN_TD_MONTHNAME}, + {"NOW", SQL_FN_TD_NOW}, + {"QUARTER", SQL_FN_TD_QUARTER}, + {"SECOND", SQL_FN_TD_SECOND}, + {"TIMESTAMPADD", SQL_FN_TD_TIMESTAMPADD}, + {"TIMESTAMPDIFF", SQL_FN_TD_TIMESTAMPDIFF}, + {"WEEK", SQL_FN_TD_WEEK}, + {"YEAR", SQL_FN_TD_YEAR}, + // Additional functions in ODBC but not Calcite: + {"CURRENT_DATE", SQL_FN_TD_CURRENT_DATE}, + {"CURRENT_TIME", SQL_FN_TD_CURRENT_TIME}, + {"CURRENT_TIMESTAMP", SQL_FN_TD_CURRENT_TIMESTAMP}, + {"EXTRACT", SQL_FN_TD_EXTRACT}}; + +static const std::unordered_map<std::string, uint32_t> string_functions = { + {"ASCII", SQL_FN_STR_ASCII}, + {"CHAR", SQL_FN_STR_CHAR}, + {"CONCAT", SQL_FN_STR_CONCAT}, + {"DIFFERENCE", SQL_FN_STR_DIFFERENCE}, + {"INSERT", SQL_FN_STR_INSERT}, + {"LCASE", SQL_FN_STR_LCASE}, + {"LEFT", SQL_FN_STR_LEFT}, + {"LENGTH", SQL_FN_STR_LENGTH}, + {"LOCATE", SQL_FN_STR_LOCATE}, + {"LTRIM", SQL_FN_STR_LTRIM}, + {"REPEAT", SQL_FN_STR_REPEAT}, + {"REPLACE", SQL_FN_STR_REPLACE}, + {"RIGHT", SQL_FN_STR_RIGHT}, + {"RTRIM", SQL_FN_STR_RTRIM}, + {"SOUNDEX", SQL_FN_STR_SOUNDEX}, + {"SPACE", SQL_FN_STR_SPACE}, + {"SUBSTRING", SQL_FN_STR_SUBSTRING}, + {"UCASE", SQL_FN_STR_UCASE}, + // Additional functions in ODBC but not Calcite: + {"LOCATE_2", SQL_FN_STR_LOCATE_2}, + {"BIT_LENGTH", SQL_FN_STR_BIT_LENGTH}, + {"CHAR_LENGTH", SQL_FN_STR_CHAR_LENGTH}, + {"CHARACTER_LENGTH", SQL_FN_STR_CHARACTER_LENGTH}, + {"OCTET_LENGTH", SQL_FN_STR_OCTET_LENGTH}, + {"POSTION", SQL_FN_STR_POSITION}, + {"SOUNDEX", SQL_FN_STR_SOUNDEX}}; +} // namespace + +void ReportSystemFunction(const std::string& function, uint32_t& current_sys_functions, + uint32_t& current_convert_functions) { + const auto& result = system_functions.find(function); + if (result != system_functions.end()) { + current_sys_functions |= result->second; + } else if (function == "CONVERT") { + // CAST and CONVERT are system functions from FlightSql/Calcite, but are Review Comment: AFAIK Flight SQL doesn't say anything about the actual SQL dialect? ########## cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/diagnostics.h: ########## @@ -0,0 +1,109 @@ +// 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. + +#pragma once + +#include <memory> +#include <string> +#include <vector> + +#include <arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/exceptions.h> +#include <arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/types.h> + +namespace driver { +namespace odbcabstraction { +class Diagnostics { + public: + struct DiagnosticsRecord { + std::string msg_text_; Review Comment: This is a struct, so the fields are public - no need to suffix with _ ########## cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/blocking_queue.h: ########## @@ -0,0 +1,131 @@ +// 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. + +#pragma once + +#include <atomic> +#include <boost/optional.hpp> +#include <condition_variable> +#include <mutex> +#include <thread> +#include <vector> + +namespace driver { +namespace odbcabstraction { + +template <typename T> +class BlockingQueue { + size_t capacity_; + std::vector<T> buffer_; + size_t buffer_size_{0}; + size_t left_{0}; // index where variables are put inside of buffer (produced) + size_t right_{0}; // index where variables are removed from buffer (consumed) + + std::mutex mtx_; + std::condition_variable not_empty_; + std::condition_variable not_full_; + + std::vector<std::thread> threads_; + std::atomic<size_t> active_threads_{0}; + std::atomic<bool> closed_{false}; + + public: + typedef std::function<boost::optional<T>(void)> Supplier; + + explicit BlockingQueue(size_t capacity) : capacity_(capacity), buffer_(capacity) {} + + void AddProducer(Supplier supplier) { + active_threads_++; + threads_.emplace_back([=] { + while (!closed_) { + // Block while queue is full + std::unique_lock<std::mutex> unique_lock(mtx_); + if (!WaitUntilCanPushOrClosed(unique_lock)) break; + unique_lock.unlock(); Review Comment: Why unlock here, when Push has to lock again below? (Pass push the guard instead?) ########## cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/blocking_queue.h: ########## @@ -0,0 +1,131 @@ +// 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. + +#pragma once + +#include <atomic> +#include <boost/optional.hpp> +#include <condition_variable> +#include <mutex> +#include <thread> +#include <vector> + +namespace driver { +namespace odbcabstraction { + +template <typename T> +class BlockingQueue { + size_t capacity_; + std::vector<T> buffer_; + size_t buffer_size_{0}; + size_t left_{0}; // index where variables are put inside of buffer (produced) + size_t right_{0}; // index where variables are removed from buffer (consumed) + + std::mutex mtx_; + std::condition_variable not_empty_; + std::condition_variable not_full_; + + std::vector<std::thread> threads_; + std::atomic<size_t> active_threads_{0}; + std::atomic<bool> closed_{false}; + + public: + typedef std::function<boost::optional<T>(void)> Supplier; + + explicit BlockingQueue(size_t capacity) : capacity_(capacity), buffer_(capacity) {} + + void AddProducer(Supplier supplier) { + active_threads_++; + threads_.emplace_back([=] { + while (!closed_) { + // Block while queue is full + std::unique_lock<std::mutex> unique_lock(mtx_); + if (!WaitUntilCanPushOrClosed(unique_lock)) break; + unique_lock.unlock(); + + // Only one thread at a time be notified and call supplier Review Comment: If the basis for this is the use of `notify_one`, that doesn't actually work AFAIK ########## cpp/src/arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.cc: ########## @@ -0,0 +1,140 @@ +// 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 "arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.h" + +#include "arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h" + +#include <sqlext.h> +#include <string> +#include <unordered_map> + +namespace driver { +namespace flight_sql { + +// The list of functions that can be converted from string to ODBC bitmasks is +// based on Calcite's SqlJdbcFunctionCall class. + +namespace { +static const std::unordered_map<std::string, uint32_t> numeric_functions = { + {"ABS", SQL_FN_NUM_ABS}, {"ACOS", SQL_FN_NUM_ACOS}, + {"ASIN", SQL_FN_NUM_ASIN}, {"ATAN", SQL_FN_NUM_ATAN}, + {"ATAN2", SQL_FN_NUM_ATAN2}, {"CEILING", SQL_FN_NUM_CEILING}, + {"COS", SQL_FN_NUM_ACOS}, {"COT", SQL_FN_NUM_COT}, + {"DEGREES", SQL_FN_NUM_DEGREES}, {"EXP", SQL_FN_NUM_EXP}, + {"FLOOR", SQL_FN_NUM_FLOOR}, {"LOG", SQL_FN_NUM_LOG}, + {"LOG10", SQL_FN_NUM_LOG10}, {"MOD", SQL_FN_NUM_MOD}, + {"PI", SQL_FN_NUM_PI}, {"POWER", SQL_FN_NUM_POWER}, + {"RADIANS", SQL_FN_NUM_RADIANS}, {"RAND", SQL_FN_NUM_RAND}, + {"ROUND", SQL_FN_NUM_ROUND}, {"SIGN", SQL_FN_NUM_SIGN}, + {"SIN", SQL_FN_NUM_SIN}, {"SQRT", SQL_FN_NUM_SQRT}, + {"TAN", SQL_FN_NUM_TAN}, {"TRUNCATE", SQL_FN_NUM_TRUNCATE}}; + +static const std::unordered_map<std::string, uint32_t> system_functions = { + {"DATABASE", SQL_FN_SYS_DBNAME}, + {"IFNULL", SQL_FN_SYS_IFNULL}, + {"USER", SQL_FN_SYS_USERNAME}}; + +static const std::unordered_map<std::string, uint32_t> datetime_functions = { + {"CURDATE", SQL_FN_TD_CURDATE}, + {"CURTIME", SQL_FN_TD_CURTIME}, + {"DAYNAME", SQL_FN_TD_DAYNAME}, + {"DAYOFMONTH", SQL_FN_TD_DAYOFMONTH}, + {"DAYOFWEEK", SQL_FN_TD_DAYOFWEEK}, + {"DAYOFYEAR", SQL_FN_TD_DAYOFYEAR}, + {"HOUR", SQL_FN_TD_HOUR}, + {"MINUTE", SQL_FN_TD_MINUTE}, + {"MONTH", SQL_FN_TD_MONTH}, + {"MONTHNAME", SQL_FN_TD_MONTHNAME}, + {"NOW", SQL_FN_TD_NOW}, + {"QUARTER", SQL_FN_TD_QUARTER}, + {"SECOND", SQL_FN_TD_SECOND}, + {"TIMESTAMPADD", SQL_FN_TD_TIMESTAMPADD}, + {"TIMESTAMPDIFF", SQL_FN_TD_TIMESTAMPDIFF}, + {"WEEK", SQL_FN_TD_WEEK}, + {"YEAR", SQL_FN_TD_YEAR}, + // Additional functions in ODBC but not Calcite: + {"CURRENT_DATE", SQL_FN_TD_CURRENT_DATE}, + {"CURRENT_TIME", SQL_FN_TD_CURRENT_TIME}, + {"CURRENT_TIMESTAMP", SQL_FN_TD_CURRENT_TIMESTAMP}, + {"EXTRACT", SQL_FN_TD_EXTRACT}}; + +static const std::unordered_map<std::string, uint32_t> string_functions = { + {"ASCII", SQL_FN_STR_ASCII}, + {"CHAR", SQL_FN_STR_CHAR}, + {"CONCAT", SQL_FN_STR_CONCAT}, + {"DIFFERENCE", SQL_FN_STR_DIFFERENCE}, + {"INSERT", SQL_FN_STR_INSERT}, + {"LCASE", SQL_FN_STR_LCASE}, + {"LEFT", SQL_FN_STR_LEFT}, + {"LENGTH", SQL_FN_STR_LENGTH}, + {"LOCATE", SQL_FN_STR_LOCATE}, + {"LTRIM", SQL_FN_STR_LTRIM}, + {"REPEAT", SQL_FN_STR_REPEAT}, + {"REPLACE", SQL_FN_STR_REPLACE}, + {"RIGHT", SQL_FN_STR_RIGHT}, + {"RTRIM", SQL_FN_STR_RTRIM}, + {"SOUNDEX", SQL_FN_STR_SOUNDEX}, + {"SPACE", SQL_FN_STR_SPACE}, + {"SUBSTRING", SQL_FN_STR_SUBSTRING}, + {"UCASE", SQL_FN_STR_UCASE}, + // Additional functions in ODBC but not Calcite: Review Comment: What does Calcite have to do with this? (Or is this perhaps a Dremio implementation detail leaking out?) -- 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: [email protected] For queries about this service, please contact Infrastructure at: [email protected]
