lidavidm commented on code in PR #40939:
URL: https://github.com/apache/arrow/pull/40939#discussion_r2101365880


##########
cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/spi/statement.h:
##########
@@ -0,0 +1,193 @@
+// 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 <boost/optional.hpp>
+#include <boost/variant.hpp>
+#include <map>
+#include <vector>
+
+namespace driver {
+namespace odbcabstraction {
+
+using boost::optional;
+
+class ResultSet;
+
+class ResultSetMetadata;
+
+/// \brief High-level representation of an ODBC statement.
+class Statement {
+ protected:
+  Statement() = default;
+
+ public:
+  virtual ~Statement() = default;
+
+  /// \brief Statement attributes that can be called at anytime.
+  ////TODO: Document attributes
+  enum StatementAttributeId {

Review Comment:
   enum class



##########
cpp/src/arrow/flight/sql/odbc/flight_sql/get_info_cache.cc:
##########
@@ -0,0 +1,1345 @@
+// 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/get_info_cache.h"
+
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h"
+
+#include <sql.h>
+#include <sqlext.h>
+#include "arrow/array.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/flight/sql/api.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/exceptions.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+
+#include "arrow/flight/sql/odbc/flight_sql/flight_sql_stream_chunk_buffer.h"
+#include "arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.h"
+#include "arrow/flight/sql/odbc/flight_sql/utils.h"
+
+// Aliases for entries in SqlInfoOptions::SqlInfo that are defined here
+// due to causing compilation errors conflicting with ODBC definitions.
+#define ARROW_SQL_IDENTIFIER_CASE 503
+#define ARROW_SQL_IDENTIFIER_QUOTE_CHAR 504
+#define ARROW_SQL_QUOTED_IDENTIFIER_CASE 505
+#define ARROW_SQL_KEYWORDS 508
+#define ARROW_SQL_NUMERIC_FUNCTIONS 509
+#define ARROW_SQL_STRING_FUNCTIONS 510
+#define ARROW_SQL_SYSTEM_FUNCTIONS 511
+#define ARROW_SQL_SCHEMA_TERM 529
+#define ARROW_SQL_PROCEDURE_TERM 530
+#define ARROW_SQL_CATALOG_TERM 531
+#define ARROW_SQL_MAX_COLUMNS_IN_GROUP_BY 544
+#define ARROW_SQL_MAX_COLUMNS_IN_INDEX 545
+#define ARROW_SQL_MAX_COLUMNS_IN_ORDER_BY 546
+#define ARROW_SQL_MAX_COLUMNS_IN_SELECT 547
+#define ARROW_SQL_MAX_COLUMNS_IN_TABLE 548
+#define ARROW_SQL_MAX_ROW_SIZE 555
+#define ARROW_SQL_MAX_TABLES_IN_SELECT 560
+
+#define ARROW_CONVERT_BIGINT 0
+#define ARROW_CONVERT_BINARY 1
+#define ARROW_CONVERT_BIT 2
+#define ARROW_CONVERT_CHAR 3
+#define ARROW_CONVERT_DATE 4
+#define ARROW_CONVERT_DECIMAL 5
+#define ARROW_CONVERT_FLOAT 6
+#define ARROW_CONVERT_INTEGER 7
+#define ARROW_CONVERT_INTERVAL_DAY_TIME 8
+#define ARROW_CONVERT_INTERVAL_YEAR_MONTH 9
+#define ARROW_CONVERT_LONGVARBINARY 10
+#define ARROW_CONVERT_LONGVARCHAR 11
+#define ARROW_CONVERT_NUMERIC 12
+#define ARROW_CONVERT_REAL 13
+#define ARROW_CONVERT_SMALLINT 14
+#define ARROW_CONVERT_TIME 15
+#define ARROW_CONVERT_TIMESTAMP 16
+#define ARROW_CONVERT_TINYINT 17
+#define ARROW_CONVERT_VARBINARY 18
+#define ARROW_CONVERT_VARCHAR 19
+
+namespace {
+// Return the corresponding field in SQLGetInfo's SQL_CONVERT_* field
+// types for the given Arrow SqlConvert enum value.
+//
+// The caller is responsible for casting the result to a uint16. Note
+// that -1 is returned if there's no corresponding entry.
+int32_t GetInfoTypeForArrowConvertEntry(int32_t convert_entry) {
+  switch (convert_entry) {
+    case ARROW_CONVERT_BIGINT:
+      return SQL_CONVERT_BIGINT;
+    case ARROW_CONVERT_BINARY:
+      return SQL_CONVERT_BINARY;
+    case ARROW_CONVERT_BIT:
+      return SQL_CONVERT_BIT;
+    case ARROW_CONVERT_CHAR:
+      return SQL_CONVERT_CHAR;
+    case ARROW_CONVERT_DATE:
+      return SQL_CONVERT_DATE;
+    case ARROW_CONVERT_DECIMAL:
+      return SQL_CONVERT_DECIMAL;
+    case ARROW_CONVERT_FLOAT:
+      return SQL_CONVERT_FLOAT;
+    case ARROW_CONVERT_INTEGER:
+      return SQL_CONVERT_INTEGER;
+    case ARROW_CONVERT_INTERVAL_DAY_TIME:
+      return SQL_CONVERT_INTERVAL_DAY_TIME;
+    case ARROW_CONVERT_INTERVAL_YEAR_MONTH:
+      return SQL_CONVERT_INTERVAL_YEAR_MONTH;
+    case ARROW_CONVERT_LONGVARBINARY:
+      return SQL_CONVERT_LONGVARBINARY;
+    case ARROW_CONVERT_LONGVARCHAR:
+      return SQL_CONVERT_LONGVARCHAR;
+    case ARROW_CONVERT_NUMERIC:
+      return SQL_CONVERT_NUMERIC;
+    case ARROW_CONVERT_REAL:
+      return SQL_CONVERT_REAL;
+    case ARROW_CONVERT_SMALLINT:
+      return SQL_CONVERT_SMALLINT;
+    case ARROW_CONVERT_TIME:
+      return SQL_CONVERT_TIME;
+    case ARROW_CONVERT_TIMESTAMP:
+      return SQL_CONVERT_TIMESTAMP;
+    case ARROW_CONVERT_TINYINT:
+      return SQL_CONVERT_TINYINT;
+    case ARROW_CONVERT_VARBINARY:
+      return SQL_CONVERT_VARBINARY;
+    case ARROW_CONVERT_VARCHAR:
+      return SQL_CONVERT_VARCHAR;
+  }
+  // Arbitrarily return a negative value
+  return -1;

Review Comment:
   Is this correct? Should wee error instead?



##########
cpp/src/arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/spi/connection.h:
##########
@@ -0,0 +1,102 @@
+// 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 <boost/algorithm/string.hpp>
+#include <boost/optional.hpp>
+#include <boost/variant.hpp>
+#include <functional>
+#include <map>
+#include <string>
+#include <vector>
+
+#include 
<arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/diagnostics.h>
+#include 
<arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/types.h>
+
+namespace driver {
+namespace odbcabstraction {
+
+/// \brief Case insensitive comparator
+struct CaseInsensitiveComparator {
+  bool operator()(const std::string_view& s1, const std::string_view& s2) 
const {
+    return boost::lexicographical_compare(s1, s2, boost::is_iless());
+  }
+};
+
+// PropertyMap is case-insensitive for keys.
+typedef std::map<std::string_view, std::string, CaseInsensitiveComparator> 
PropertyMap;
+
+class Statement;
+
+/// \brief High-level representation of an ODBC connection.
+class Connection {
+ protected:
+  Connection() = default;
+
+ public:
+  virtual ~Connection() = default;
+
+  /// \brief Connection attributes
+  enum AttributeId {
+    ACCESS_MODE,         // uint32_t - Tells if it should support write 
operations
+    CONNECTION_DEAD,     // uint32_t - Tells if connection is still alive
+    CONNECTION_TIMEOUT,  // uint32_t - The timeout for connection functions 
after
+                         // connecting.
+    CURRENT_CATALOG,     // std::string - The current catalog
+    LOGIN_TIMEOUT,       // uint32_t - The timeout for the initial connection
+    PACKET_SIZE,         // uint32_t - The Packet Size
+  };
+
+  typedef boost::variant<std::string, void*, uint64_t, uint32_t> Attribute;

Review Comment:
   Use std::variant



##########
cpp/src/arrow/flight/sql/odbc/flight_sql/include/flight_sql/config/configuration.h:
##########
@@ -0,0 +1,77 @@
+// 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 <stdint.h>
+#include <string>
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/spi/connection.h"
+
+// winuser.h needs to be included after windows.h, which is defined in 
platform.h
+#include <winuser.h>
+namespace driver {
+namespace flight_sql {
+namespace config {
+
+#define TRUE_STR "true"

Review Comment:
   ...is there any actual value to having these constants?



##########
cpp/src/arrow/flight/sql/odbc/odbcabstraction/odbc_impl/odbc_connection.cc:
##########
@@ -0,0 +1,777 @@
+// 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/odbcabstraction/include/odbcabstraction/odbc_impl/odbc_connection.h"
+
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/exceptions.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/odbc_impl/attribute_utils.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/odbc_impl/odbc_descriptor.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/odbc_impl/odbc_environment.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/odbc_impl/odbc_statement.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/spi/connection.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/spi/statement.h"
+
+#include <odbcinst.h>
+#include <sql.h>
+#include <sqlext.h>
+#include <boost/algorithm/string.hpp>
+#include <boost/xpressive/xpressive.hpp>
+#include <iterator>
+#include <memory>
+#include <utility>
+
+using ODBC::ODBCConnection;
+using ODBC::ODBCDescriptor;
+using ODBC::ODBCStatement;
+
+using driver::odbcabstraction::Connection;
+using driver::odbcabstraction::Diagnostics;
+using driver::odbcabstraction::DriverException;
+using driver::odbcabstraction::Statement;
+
+namespace {
+// Key-value pairs separated by semi-colon.
+// Note that the value can be wrapped in curly braces to escape other 
significant
+// characters such as semi-colons and equals signs. NOTE: This can be 
optimized to be
+// built statically.
+const boost::xpressive::sregex CONNECTION_STR_REGEX(
+    boost::xpressive::sregex::compile("([^=;]+)=({.+}|[^=;]+|[^;])"));

Review Comment:
   Can we use stdlib regex?



##########
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.

Review Comment:
   Use 
[`[[fallthrough]]`](https://en.cppreference.com/w/cpp/language/attributes/fallthrough)



##########
cpp/src/arrow/flight/sql/odbc/flight_sql/flight_sql_stream_chunk_buffer.cc:
##########
@@ -0,0 +1,69 @@
+// 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_stream_chunk_buffer.h"
+#include "arrow/flight/sql/odbc/flight_sql/utils.h"
+
+namespace driver {
+namespace flight_sql {
+
+using arrow::flight::FlightEndpoint;
+
+FlightStreamChunkBuffer::FlightStreamChunkBuffer(
+    FlightSqlClient& flight_sql_client,
+    const arrow::flight::FlightCallOptions& call_options,
+    const std::shared_ptr<FlightInfo>& flight_info, size_t queue_capacity)
+    : queue_(queue_capacity) {
+  // FIXME: Endpoint iteration should consider endpoints may be at different 
hosts

Review Comment:
   File an issue for this?



##########
cpp/src/arrow/flight/sql/odbc/flight_sql/get_info_cache.cc:
##########
@@ -0,0 +1,1345 @@
+// 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/get_info_cache.h"
+
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/platform.h"
+
+#include <sql.h>
+#include <sqlext.h>
+#include "arrow/array.h"
+#include "arrow/array/array_nested.h"
+#include "arrow/flight/sql/api.h"
+#include 
"arrow/flight/sql/odbc/odbcabstraction/include/odbcabstraction/exceptions.h"
+#include "arrow/scalar.h"
+#include "arrow/type_fwd.h"
+
+#include "arrow/flight/sql/odbc/flight_sql/flight_sql_stream_chunk_buffer.h"
+#include "arrow/flight/sql/odbc/flight_sql/scalar_function_reporter.h"
+#include "arrow/flight/sql/odbc/flight_sql/utils.h"
+
+// Aliases for entries in SqlInfoOptions::SqlInfo that are defined here
+// due to causing compilation errors conflicting with ODBC definitions.
+#define ARROW_SQL_IDENTIFIER_CASE 503
+#define ARROW_SQL_IDENTIFIER_QUOTE_CHAR 504
+#define ARROW_SQL_QUOTED_IDENTIFIER_CASE 505
+#define ARROW_SQL_KEYWORDS 508
+#define ARROW_SQL_NUMERIC_FUNCTIONS 509
+#define ARROW_SQL_STRING_FUNCTIONS 510
+#define ARROW_SQL_SYSTEM_FUNCTIONS 511
+#define ARROW_SQL_SCHEMA_TERM 529
+#define ARROW_SQL_PROCEDURE_TERM 530
+#define ARROW_SQL_CATALOG_TERM 531
+#define ARROW_SQL_MAX_COLUMNS_IN_GROUP_BY 544
+#define ARROW_SQL_MAX_COLUMNS_IN_INDEX 545
+#define ARROW_SQL_MAX_COLUMNS_IN_ORDER_BY 546
+#define ARROW_SQL_MAX_COLUMNS_IN_SELECT 547
+#define ARROW_SQL_MAX_COLUMNS_IN_TABLE 548
+#define ARROW_SQL_MAX_ROW_SIZE 555
+#define ARROW_SQL_MAX_TABLES_IN_SELECT 560
+
+#define ARROW_CONVERT_BIGINT 0
+#define ARROW_CONVERT_BINARY 1
+#define ARROW_CONVERT_BIT 2
+#define ARROW_CONVERT_CHAR 3
+#define ARROW_CONVERT_DATE 4
+#define ARROW_CONVERT_DECIMAL 5
+#define ARROW_CONVERT_FLOAT 6
+#define ARROW_CONVERT_INTEGER 7
+#define ARROW_CONVERT_INTERVAL_DAY_TIME 8
+#define ARROW_CONVERT_INTERVAL_YEAR_MONTH 9
+#define ARROW_CONVERT_LONGVARBINARY 10
+#define ARROW_CONVERT_LONGVARCHAR 11
+#define ARROW_CONVERT_NUMERIC 12
+#define ARROW_CONVERT_REAL 13
+#define ARROW_CONVERT_SMALLINT 14
+#define ARROW_CONVERT_TIME 15
+#define ARROW_CONVERT_TIMESTAMP 16
+#define ARROW_CONVERT_TINYINT 17
+#define ARROW_CONVERT_VARBINARY 18
+#define ARROW_CONVERT_VARCHAR 19
+
+namespace {
+// Return the corresponding field in SQLGetInfo's SQL_CONVERT_* field
+// types for the given Arrow SqlConvert enum value.
+//
+// The caller is responsible for casting the result to a uint16. Note
+// that -1 is returned if there's no corresponding entry.
+int32_t GetInfoTypeForArrowConvertEntry(int32_t convert_entry) {
+  switch (convert_entry) {
+    case ARROW_CONVERT_BIGINT:
+      return SQL_CONVERT_BIGINT;
+    case ARROW_CONVERT_BINARY:
+      return SQL_CONVERT_BINARY;
+    case ARROW_CONVERT_BIT:
+      return SQL_CONVERT_BIT;
+    case ARROW_CONVERT_CHAR:
+      return SQL_CONVERT_CHAR;
+    case ARROW_CONVERT_DATE:
+      return SQL_CONVERT_DATE;
+    case ARROW_CONVERT_DECIMAL:
+      return SQL_CONVERT_DECIMAL;
+    case ARROW_CONVERT_FLOAT:
+      return SQL_CONVERT_FLOAT;
+    case ARROW_CONVERT_INTEGER:
+      return SQL_CONVERT_INTEGER;
+    case ARROW_CONVERT_INTERVAL_DAY_TIME:
+      return SQL_CONVERT_INTERVAL_DAY_TIME;
+    case ARROW_CONVERT_INTERVAL_YEAR_MONTH:
+      return SQL_CONVERT_INTERVAL_YEAR_MONTH;
+    case ARROW_CONVERT_LONGVARBINARY:
+      return SQL_CONVERT_LONGVARBINARY;
+    case ARROW_CONVERT_LONGVARCHAR:
+      return SQL_CONVERT_LONGVARCHAR;
+    case ARROW_CONVERT_NUMERIC:
+      return SQL_CONVERT_NUMERIC;
+    case ARROW_CONVERT_REAL:
+      return SQL_CONVERT_REAL;
+    case ARROW_CONVERT_SMALLINT:
+      return SQL_CONVERT_SMALLINT;
+    case ARROW_CONVERT_TIME:
+      return SQL_CONVERT_TIME;
+    case ARROW_CONVERT_TIMESTAMP:
+      return SQL_CONVERT_TIMESTAMP;
+    case ARROW_CONVERT_TINYINT:
+      return SQL_CONVERT_TINYINT;
+    case ARROW_CONVERT_VARBINARY:
+      return SQL_CONVERT_VARBINARY;
+    case ARROW_CONVERT_VARCHAR:
+      return SQL_CONVERT_VARCHAR;
+  }
+  // Arbitrarily return a negative value
+  return -1;
+}
+
+// Return the corresponding bitmask to OR in SQLGetInfo's SQL_CONVERT_* field
+// value for the given Arrow SqlConvert enum value.
+//
+// This is _not_ a bit position, it is an integer with only a single bit set.
+uint32_t GetCvtBitForArrowConvertEntry(int32_t convert_entry) {
+  switch (convert_entry) {
+    case ARROW_CONVERT_BIGINT:
+      return SQL_CVT_BIGINT;
+    case ARROW_CONVERT_BINARY:
+      return SQL_CVT_BINARY;
+    case ARROW_CONVERT_BIT:
+      return SQL_CVT_BIT;
+    case ARROW_CONVERT_CHAR:
+      return SQL_CVT_CHAR | SQL_CVT_WCHAR;
+    case ARROW_CONVERT_DATE:
+      return SQL_CVT_DATE;
+    case ARROW_CONVERT_DECIMAL:
+      return SQL_CVT_DECIMAL;
+    case ARROW_CONVERT_FLOAT:
+      return SQL_CVT_FLOAT;
+    case ARROW_CONVERT_INTEGER:
+      return SQL_CVT_INTEGER;
+    case ARROW_CONVERT_INTERVAL_DAY_TIME:
+      return SQL_CVT_INTERVAL_DAY_TIME;
+    case ARROW_CONVERT_INTERVAL_YEAR_MONTH:
+      return SQL_CVT_INTERVAL_YEAR_MONTH;
+    case ARROW_CONVERT_LONGVARBINARY:
+      return SQL_CVT_LONGVARBINARY;
+    case ARROW_CONVERT_LONGVARCHAR:
+      return SQL_CVT_LONGVARCHAR | SQL_CVT_WLONGVARCHAR;
+    case ARROW_CONVERT_NUMERIC:
+      return SQL_CVT_NUMERIC;
+    case ARROW_CONVERT_REAL:
+      return SQL_CVT_REAL;
+    case ARROW_CONVERT_SMALLINT:
+      return SQL_CVT_SMALLINT;
+    case ARROW_CONVERT_TIME:
+      return SQL_CVT_TIME;
+    case ARROW_CONVERT_TIMESTAMP:
+      return SQL_CVT_TIMESTAMP;
+    case ARROW_CONVERT_TINYINT:
+      return SQL_CVT_TINYINT;
+    case ARROW_CONVERT_VARBINARY:
+      return SQL_CVT_VARBINARY;
+    case ARROW_CONVERT_VARCHAR:
+      return SQL_CVT_VARCHAR | SQL_CVT_WLONGVARCHAR;
+  }
+  // Note: GUID not supported by GetSqlInfo.
+  // Return zero, which has no bits set.
+  return 0;
+}
+
+inline int32_t ScalarToInt32(arrow::UnionScalar* scalar) {
+  return 
reinterpret_cast<arrow::Int32Scalar*>(scalar->child_value().get())->value;

Review Comment:
   checked_cast?
   
   Also, why `inline`?



-- 
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]

Reply via email to