You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by GitBox <gi...@apache.org> on 2022/01/10 20:57:12 UTC

[GitHub] [arrow] lidavidm commented on a change in pull request #11999: Add missing metadata on Arrow schemas returned by Flight SQL

lidavidm commented on a change in pull request #11999:
URL: https://github.com/apache/arrow/pull/11999#discussion_r781538101



##########
File path: cpp/src/arrow/flight/sql/column_metadata.cc
##########
@@ -0,0 +1,187 @@
+// 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/column_metadata.h"
+
+#include <utility>
+
+namespace arrow {
+namespace flight {
+namespace sql {
+namespace {
+  /// \brief Constant variable used to convert boolean true value
+  ///        to a string.
+  const char* BOOLEAN_TRUE_STR = "YES";
+  /// \brief Constant variable used to convert boolean false value
+  ///        to a string.
+  const char* BOOLEAN_FALSE_STR = "NO";
+
+  std::string BooleanToString(bool boolean_value) {
+    return boolean_value ? BOOLEAN_TRUE_STR :
+           BOOLEAN_FALSE_STR;
+  }
+
+  bool StringToBoolean(const std::string& string_value) {
+    return string_value == BOOLEAN_TRUE_STR;
+  }
+}  // namespace
+
+const char* ColumnMetadata::kCatalogName = "ARROW:FLIGHT:SQL:CATALOG_NAME";
+const char* ColumnMetadata::kSchemaName = "ARROW:FLIGHT:SQL:SCHEMA_NAME";
+const char* ColumnMetadata::kTableName = "ARROW:FLIGHT:SQL:TABLE_NAME";
+const char* ColumnMetadata::kPrecision = "ARROW:FLIGHT:SQL:PRECISION";
+const char* ColumnMetadata::kScale = "ARROW:FLIGHT:SQL:SCALE";
+const char* ColumnMetadata::kIsAutoIncrement = "ARROW:FLIGHT:SQL:IS_AUTO_INCREMENT";
+const char* ColumnMetadata::kIsCaseSensitive = "ARROW:FLIGHT:SQL:IS_CASE_SENSITIVE";
+const char* ColumnMetadata::kIsReadOnly = "ARROW:FLIGHT:SQL:IS_READ_ONLY";
+const char* ColumnMetadata::kIsSearchable = "ARROW:FLIGHT:SQL:IS_SEARCHABLE";
+
+ColumnMetadata::ColumnMetadata(std::shared_ptr<arrow::KeyValueMetadata> metadata_map) :
+  metadata_map_(std::move(metadata_map)) {
+}
+
+arrow::Result<std::string> ColumnMetadata::GetCatalogName() const {
+  return metadata_map_->Get(kCatalogName);
+}
+
+arrow::Result<std::string> ColumnMetadata::GetSchemaName() const {
+  return metadata_map_->Get(kSchemaName);
+}
+
+arrow::Result<std::string> ColumnMetadata::GetTableName() const {
+  return metadata_map_->Get(kTableName);
+}
+
+arrow::Result<int32_t> ColumnMetadata::GetPrecision() const {
+  const Result <std::string> &result = metadata_map_->Get(kPrecision);
+  std::string precision_string;
+  ARROW_ASSIGN_OR_RAISE(precision_string, result);
+
+  return std::stoi(precision_string);
+}
+
+arrow::Result<int32_t> ColumnMetadata::GetScale() const {
+  std::string scale_string;
+  ARROW_ASSIGN_OR_RAISE(scale_string, metadata_map_->Get(kScale));
+
+  return std::stoi(scale_string);
+}
+
+arrow::Result<bool> ColumnMetadata::GetIsAutoIncrement() const {
+  std::string auto_increment_string;
+  ARROW_ASSIGN_OR_RAISE(auto_increment_string, metadata_map_->Get(kIsAutoIncrement));
+  return StringToBoolean(auto_increment_string);
+}
+
+arrow::Result<bool> ColumnMetadata::GetIsCaseSensitive() const {
+  std::string is_case_sensitive;
+  ARROW_ASSIGN_OR_RAISE(is_case_sensitive, metadata_map_->Get(kIsAutoIncrement));
+  return StringToBoolean(is_case_sensitive);
+}
+
+arrow::Result<bool> ColumnMetadata::GetIsReadOnly() const {
+  std::string is_read_only;
+  ARROW_ASSIGN_OR_RAISE(is_read_only, metadata_map_->Get(kIsAutoIncrement));
+  return StringToBoolean(is_read_only);
+}
+
+arrow::Result<bool> ColumnMetadata::GetIsSearchable() const {
+  std::string is_case_sensitive;
+  ARROW_ASSIGN_OR_RAISE(is_case_sensitive, metadata_map_->Get(kIsAutoIncrement));
+  return StringToBoolean(is_case_sensitive);
+}
+
+ColumnMetadata::ColumnMetadataBuilder ColumnMetadata::Builder() {
+  const ColumnMetadataBuilder &builder = ColumnMetadataBuilder{};
+  return builder;

Review comment:
       nit, but this could just be `return {};` I think (or `return ColumnMetadataBuilder()`).

##########
File path: format/FlightSql.proto
##########
@@ -933,6 +933,16 @@ message CommandGetDbSchemas {
  *  [optional] table_schema: bytes not null (schema of the table as described in Schema.fbs::Schema,
  *                                           it is serialized as an IPC message.)
  * >
+ * Fields on table_schema may contain the following metadata:
+ *  - ARROW:FLIGHT:SQL:CATALOG_NAME      - Table's catalog name
+ *  - ARROW:FLIGHT:SQL:DB_SCHEMA_NAME    - Database schema name
+ *  - ARROW:FLIGHT:SQL:TABLE_NAME        - Table name
+ *  - ARROW:FLIGHT:SQL:PRECISION         - Column precision/size
+ *  - ARROW:FLIGHT:SQL:SCALE             - Column scale/decimal digits
+ *  - ARROW:FLIGHT:SQL:IS_AUTO_INCREMENT - "1" indicates if the column is auto incremented, "0" otherwise.

Review comment:
       The source seems to use "YES" and "NO" instead of "1" and "0".

##########
File path: cpp/src/arrow/flight/sql/column_metadata.h
##########
@@ -0,0 +1,168 @@
+// 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 <string>
+
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief Helper class to set column metadata.
+class ColumnMetadata {
+ private:
+  std::shared_ptr<arrow::KeyValueMetadata> metadata_map_;
+  explicit ColumnMetadata(std::shared_ptr<arrow::KeyValueMetadata> metadata_map);
+
+ public:
+  class ColumnMetadataBuilder;
+
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kCatalogName;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kSchemaName;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kTableName;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kPrecision;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kScale;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsAutoIncrement;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsCaseSensitive;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsReadOnly;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsSearchable;
+
+  /// \brief Static initializer.
+  static ColumnMetadataBuilder Builder();
+
+  /// \brief  Return the catalog name set in the KeyValueMetadata.
+  /// \return The catalog name.
+  arrow::Result<std::string> GetCatalogName() const;
+
+  /// \brief  Return the schema name set in the KeyValueMetadata.
+  /// \return The schema name.
+  arrow::Result<std::string> GetSchemaName() const;
+
+  /// \brief  Return the table name set in the KeyValueMetadata.
+  /// \return The table name.
+  arrow::Result<std::string> GetTableName() const;
+
+  /// \brief  Return the precision set in the KeyValueMetadata.
+  /// \return The precision.
+  arrow::Result<int32_t> GetPrecision() const;
+
+  /// \brief  Return the scale set in the KeyValueMetadata.
+  /// \return The scale.
+  arrow::Result<int32_t> GetScale() const;
+
+  /// \brief  Return the IsAutoIncrement set in the KeyValueMetadata.
+  /// \return The IsAutoIncrement.
+  arrow::Result<bool> GetIsAutoIncrement() const;
+
+  /// \brief  Return the IsCaseSensitive set in the KeyValueMetadata.
+  /// \return The IsCaseSensitive.
+  arrow::Result<bool> GetIsCaseSensitive() const;
+
+  /// \brief  Return the IsReadOnly set in the KeyValueMetadata.
+  /// \return The IsReadOnly.
+  arrow::Result<bool> GetIsReadOnly() const;
+
+  /// \brief  Return the IsSearchable set in the KeyValueMetadata.
+  /// \return The IsSearchable.
+  arrow::Result<bool> GetIsSearchable() const;
+
+  /// \brief  Return the KeyValueMetadata.
+  /// \return The KeyValueMetadata.
+  std::shared_ptr<arrow::KeyValueMetadata> GetMetadataMap() const;

Review comment:
       Hmm, this could be `const std::shared_ptr<>& metadata_map(); const` to save a shared_ptr copy & fit the naming convention better (pure getters are allowed to be snake_case)

##########
File path: cpp/src/arrow/flight/sql/column_metadata.h
##########
@@ -0,0 +1,168 @@
+// 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 <string>
+
+#include "arrow/util/key_value_metadata.h"
+
+namespace arrow {
+namespace flight {
+namespace sql {
+
+/// \brief Helper class to set column metadata.
+class ColumnMetadata {
+ private:
+  std::shared_ptr<arrow::KeyValueMetadata> metadata_map_;
+  explicit ColumnMetadata(std::shared_ptr<arrow::KeyValueMetadata> metadata_map);
+
+ public:
+  class ColumnMetadataBuilder;
+
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kCatalogName;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kSchemaName;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kTableName;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kPrecision;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kScale;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsAutoIncrement;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsCaseSensitive;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsReadOnly;
+  /// \brief Constant variable to hold the value of the key that
+  ///        will be used in the KeyValueMetadata class.
+  static const char* kIsSearchable;
+
+  /// \brief Static initializer.
+  static ColumnMetadataBuilder Builder();
+
+  /// \brief  Return the catalog name set in the KeyValueMetadata.
+  /// \return The catalog name.
+  arrow::Result<std::string> GetCatalogName() const;
+
+  /// \brief  Return the schema name set in the KeyValueMetadata.
+  /// \return The schema name.
+  arrow::Result<std::string> GetSchemaName() const;
+
+  /// \brief  Return the table name set in the KeyValueMetadata.
+  /// \return The table name.
+  arrow::Result<std::string> GetTableName() const;
+
+  /// \brief  Return the precision set in the KeyValueMetadata.
+  /// \return The precision.
+  arrow::Result<int32_t> GetPrecision() const;
+
+  /// \brief  Return the scale set in the KeyValueMetadata.
+  /// \return The scale.
+  arrow::Result<int32_t> GetScale() const;
+
+  /// \brief  Return the IsAutoIncrement set in the KeyValueMetadata.
+  /// \return The IsAutoIncrement.
+  arrow::Result<bool> GetIsAutoIncrement() const;
+
+  /// \brief  Return the IsCaseSensitive set in the KeyValueMetadata.
+  /// \return The IsCaseSensitive.
+  arrow::Result<bool> GetIsCaseSensitive() const;
+
+  /// \brief  Return the IsReadOnly set in the KeyValueMetadata.
+  /// \return The IsReadOnly.
+  arrow::Result<bool> GetIsReadOnly() const;
+
+  /// \brief  Return the IsSearchable set in the KeyValueMetadata.
+  /// \return The IsSearchable.
+  arrow::Result<bool> GetIsSearchable() const;
+
+  /// \brief  Return the KeyValueMetadata.
+  /// \return The KeyValueMetadata.
+  std::shared_ptr<arrow::KeyValueMetadata> GetMetadataMap() const;
+
+  /// \brief A builder class to construct the ColumnMetadata object.
+  class ColumnMetadataBuilder {
+    std::shared_ptr<arrow::KeyValueMetadata> metadata_map_;

Review comment:
       nit: for consistency, can we explicitly declare this `private:` and put this below the `public` declarations?




-- 
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: github-unsubscribe@arrow.apache.org

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