You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/11/01 06:09:49 UTC

[GitHub] [ignite-3] ptupitsyn commented on a diff in pull request #1287: IGNITE-17590 C++ 3.0: Implement RecordBinaryView

ptupitsyn commented on code in PR #1287:
URL: https://github.com/apache/ignite-3/pull/1287#discussion_r1010095028


##########
modules/platforms/cpp/ignite/client/table/ignite_tuple.h:
##########
@@ -0,0 +1,260 @@
+/*
+ * 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 "ignite/common/config.h"
+#include "ignite/common/ignite_error.h"
+
+#include <any>
+#include <initializer_list>
+#include <string_view>
+#include <unordered_map>
+#include <vector>
+#include <utility>
+
+namespace ignite {
+
+class ignite_tuple_builder;
+
+/**
+ * Ignite tuple.
+ */
+class ignite_tuple {
+    friend class ignite_tuple_builder;
+public:
+    // Default
+    ignite_tuple() = default;
+
+    /**
+     * Constructor.
+     *
+     * @param pairs Pairs.
+     */
+    ignite_tuple(std::initializer_list<std::pair<std::string, std::any>> pairs)
+        : m_pairs(pairs)
+        , m_indices()
+    {
+        for (size_t i = 0; i < m_pairs.size(); ++i)
+            m_indices.emplace(std::make_pair(parse_name(m_pairs[i].first), i));
+    }
+
+    /**
+     * Gets a number of columns in the tuple.
+     *
+     * @return Number of columns in the tuple.
+     */
+    [[nodiscard]] std::int32_t column_count() const noexcept {
+        return std::int32_t(m_pairs.size());
+    }
+
+    /**
+     * Gets the value of the specified column.
+     *
+     * @param idx The column index.
+     * @return Column value.
+     */
+    [[nodiscard]] const std::any& get(uint32_t idx) const {
+        if (idx > m_pairs.size()) {
+            throw ignite_error("Index is too large: idx=" + std::to_string(idx) +
+                ", columns_num=" + std::to_string(m_pairs.size()));
+        }
+        return m_pairs[idx].second;
+    }
+
+    /**
+     * Gets the value of the specified column.
+     *
+     * @tparam T Column type.
+     * @param idx The column index.
+     * @return Column value.
+     */
+    template<typename T>
+    [[nodiscard]] T get(uint32_t idx) const {
+        return std::any_cast<T>(get(idx));
+    }
+
+    /**
+     * Sets the value of the specified column.
+     *
+     * @tparam T Column type.
+     * @param idx The column index.
+     * @param value Value.
+     */
+    template<typename T>
+    void set(uint32_t idx, T&& value) {
+        if (idx > m_pairs.size()) {
+            throw ignite_error("Index is too large: idx=" + std::to_string(idx) +
+                ", columns_num=" + std::to_string(m_pairs.size()));
+        }
+        m_pairs[idx].second = std::forward<T>(value);
+    }
+
+    /**
+     * Gets the value of the specified column.
+     *
+     * @param name The column name.
+     * @return Column value.
+     */
+    [[nodiscard]] const std::any& get(std::string_view name) const {
+        auto it = m_indices.find(parse_name(name));
+        if (it == m_indices.end())
+            throw ignite_error("Can not find column with the name '" + std::string(name) + "' in the tuple");
+        auto idx = it->second;
+        return m_pairs[idx].second;
+    }
+
+    /**
+     * Gets the value of the specified column.
+     *
+     * @tparam T Column type.
+     * @param name The column name.
+     * @return Column value.
+     */
+    template<typename T>
+    [[nodiscard]] T get(std::string_view name) const {
+        return std::any_cast<T>(get(name));
+    }
+
+    /**
+     * Sets the value of the specified column.
+     *
+     * @tparam T Column type.
+     * @param name The column name.
+     * @param value Value.
+     */
+    template<typename T>
+    void set(std::string_view name, T&& value) {

Review Comment:
   We have `ignite_tuple_builder`, but the tuple itself is also mutable? I think we should pick one:
   * Builder + immutable Tuple with `to_builder` method
   * Mutable Tuple without builder (used in Java and .NET clients)



##########
modules/platforms/cpp/ignite/client/detail/table/table_impl.cpp:
##########
@@ -0,0 +1,715 @@
+/*
+ * 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 "ignite/client/detail/table/table_impl.h"
+
+#include "ignite/protocol/bitset_span.h"
+#include "ignite/protocol/reader.h"
+#include "ignite/protocol/writer.h"
+#include "ignite/schema/binary_tuple_builder.h"
+#include "ignite/schema/binary_tuple_parser.h"
+#include "ignite/common/bits.h"
+#include "ignite/common/ignite_error.h"
+
+namespace ignite::detail {
+
+/**
+ * Claim space for the column.
+ *
+ * @param builder Binary tuple builder.
+ * @param typ Column type.
+ * @param index Tuple field index.
+ * @param tuple Tuple.
+ */
+void claim_column(binary_tuple_builder& builder, ignite_type typ, std::int32_t index, const ignite_tuple& tuple) {
+    switch (typ) {
+        case ignite_type::INT8:
+            builder.claim_int8(tuple.get<std::int8_t>(index));
+            break;
+        case ignite_type::INT16:
+            builder.claim_int16(tuple.get<std::int16_t>(index));
+            break;
+        case ignite_type::INT32:
+            builder.claim_int32(tuple.get<std::int32_t>(index));
+            break;
+        case ignite_type::INT64:
+            builder.claim_int64(tuple.get<std::int64_t>(index));
+            break;
+        case ignite_type::FLOAT:
+            builder.claim_float(tuple.get<float>(index));
+            break;
+        case ignite_type::DOUBLE:
+            builder.claim_double(tuple.get<double>(index));
+            break;
+        case ignite_type::UUID:
+            builder.claim_uuid(tuple.get<uuid>(index));
+            break;
+        case ignite_type::STRING:
+            builder.claim(SizeT(tuple.get<const std::string&>(index).size()));
+            break;
+        case ignite_type::BINARY:
+            builder.claim(SizeT(tuple.get<const std::vector<std::byte>&>(index).size()));
+            break;
+        default:
+            // TODO: Support other types

Review Comment:
   Let's link a ticket.



##########
modules/platforms/cpp/ignite/client/table/record_view.h:
##########
@@ -0,0 +1,493 @@
+/*
+ * 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 "ignite/client/table/ignite_tuple.h"
+#include "ignite/client/transaction/transaction.h"
+
+#include "ignite/common/config.h"
+#include "ignite/common/ignite_result.h"
+
+#include <memory>
+#include <utility>
+#include <vector>
+#include <type_traits>
+
+namespace ignite {
+
+class table;
+
+namespace detail {
+class table_impl;
+}
+
+/**
+ * Record view interface provides methods to access table records.
+ */
+template<typename T>
+class record_view {
+public:
+    typedef typename std::decay<T>::type value_type;
+
+    // Deleted
+    record_view(const record_view &) = delete;
+    record_view &operator=(const record_view &) = delete;
+
+    // Default
+    record_view() = default;
+    ~record_view() = default;
+    record_view(record_view &&) noexcept = default;
+    record_view &operator=(record_view &&) noexcept = default;
+};
+
+/**
+ * Record view interface provides methods to access table records.
+ */
+template<>
+class record_view<ignite_tuple> {
+    friend class table;
+public:
+    typedef ignite_tuple value_type;
+
+    // Deleted
+    record_view(const record_view &) = delete;
+    record_view &operator=(const record_view &) = delete;
+
+    // Default
+    record_view() = default;
+    ~record_view() = default;
+    record_view(record_view &&) noexcept = default;
+    record_view &operator=(record_view &&) noexcept = default;
+
+    /**
+     * Gets a record by key asynchronously.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param key Key.
+     * @param callback Callback which is called on success with value if it
+     *   exists and @c std::nullopt otherwise
+     */
+    IGNITE_API void get_async(transaction* tx, const value_type& key, ignite_callback<std::optional<value_type>> callback);
+
+    /**
+     * Gets a record by key.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param key Key.
+     * @return Value if exists and @c std::nullopt otherwise.
+     */
+    [[nodiscard]] IGNITE_API std::optional<value_type> get(transaction* tx, const value_type& key) {
+        return sync<std::optional<value_type>>([this, tx, &key] (auto callback) {
+            get_async(tx, key, std::move(callback));
+        });
+    }
+
+    /**
+     * Gets multiple records by keys asynchronously.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param keys Keys.
+     * @param callback Callback that called on operation completion. Called with
+     *   resulting records with all columns filled from the table. The order of
+     *   elements is guaranteed to be the same as the order of keys. If a record
+     *   does not exist, the resulting element of the corresponding order is
+     *   @c std::nullopt.
+     */
+    IGNITE_API void get_all_async(transaction* tx, std::vector<value_type> keys,
+        ignite_callback<std::vector<std::optional<value_type>>> callback);
+
+    /**
+     * Gets multiple records by keys.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param keys Keys.
+     * @return Resulting records with all columns filled from the table.
+     *   The order of elements is guaranteed to be the same as the order of
+     *   keys. If a record does not exist, the resulting element of the
+     *   corresponding order is @c std::nullopt.
+     */
+    [[nodiscard]] IGNITE_API std::vector<std::optional<value_type>> get_all(transaction* tx, std::vector<value_type> keys) {
+        return sync<std::vector<std::optional<value_type>>>([this, tx, keys = std::move(keys)] (auto callback) mutable {
+            get_all_async(tx, std::move(keys), std::move(callback));
+        });
+    }
+
+    /**
+     * Inserts a record into the table if does not exist or replaces the existed one.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *  single operation is used.
+     * @param record A record to insert into the table. The record cannot be @c nullptr.
+     * @param callback Callback.
+     */
+    IGNITE_API void upsert_async(transaction* tx, const value_type& record, ignite_callback<void> callback);
+
+    /**
+     * Inserts a record into the table if does not exist or replaces the existed one.

Review Comment:
   Here and other places: "existed" -> "existing". I don't think "existed" can be used like this.



##########
modules/platforms/cpp/ignite/client/detail/table/table_impl.h:
##########
@@ -41,24 +45,301 @@ class table_impl {
      *
      * @param name Name.
      * @param id ID.
+     * @param connection Connection.
      */
-    table_impl(std::string name, const uuid &id)
+    table_impl(std::string name, const uuid &id, std::shared_ptr<cluster_connection> connection)
         : m_name(std::move(name))
-        , m_id(id) { }
+        , m_id(id)
+        , m_connection(std::move(connection)) { }
 
     /**
-     * Get table name.
+     * Gets table name.
      *
      * @return Table name.
      */
     [[nodiscard]] const std::string &name() const { return m_name; }
 
+    /**
+     * Gets the latest schema.
+     *
+     * @param callback Callback which is going to be called with the latest schema.
+     */
+    void get_latest_schema_async(ignite_callback<std::shared_ptr<schema>> callback);
+
+    /**
+     * Gets the latest schema.
+     *
+     * @param handler Callback to call on error during retrieval of the latest schema.
+     */
+    template <typename T>
+    void with_latest_schema_async(ignite_callback<T> handler, std::function<void(const schema&, ignite_callback<T>)> callback) {
+        get_latest_schema_async(
+            [this, handler = std::move(handler), callback = std::move(callback)]
+                (ignite_result<std::shared_ptr<schema>>&& res) mutable {
+            if (res.has_error()) {
+                handler(ignite_error{res.error()});
+                return;
+            }
+
+            auto schema = res.value();
+            if (!schema) {
+                handler(ignite_error{"Can not get a schema for the table " + m_name});

Review Comment:
   When is this the case? Server should always return a schema, unless there is an error that is already handled above.



##########
modules/platforms/cpp/ignite/client/table/record_view.cpp:
##########
@@ -0,0 +1,156 @@
+/*
+ * 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 "ignite/client/table/record_view.h"
+#include "ignite/client/detail/table/table_impl.h"
+
+namespace ignite {
+
+void record_view<ignite_tuple>::get_async(transaction *tx, const ignite_tuple& key,
+        ignite_callback<std::optional<value_type>> callback)
+{
+    if (0 == key.column_count())
+        throw ignite_error("Tuple can not be empty");
+
+    m_impl->get_async(tx, key, std::move(callback));
+}
+
+void record_view<ignite_tuple>::upsert_async(transaction *tx, const ignite_tuple& record,
+        ignite_callback<void> callback)
+{
+    if (0 == record.column_count())
+        throw ignite_error("Tuple can not be empty");
+
+    m_impl->upsert_async(tx, record, std::move(callback));
+}
+
+void record_view<ignite_tuple>::get_all_async(transaction *tx, std::vector<value_type> keys,
+        ignite_callback<std::vector<std::optional<value_type>>> callback)
+{
+    if (keys.empty())
+        throw ignite_error("At least one key should be supplied");

Review Comment:
   I think we should return empty vector when keys are empty. At least this is what Java and .NET APIs do. And it may save the user some extra checks.
   
   The same applies to all multi-record operations.



-- 
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: notifications-unsubscribe@ignite.apache.org

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