You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by "isapego (via GitHub)" <gi...@apache.org> on 2023/04/04 08:57:00 UTC

[GitHub] [ignite-3] isapego commented on a diff in pull request #1891: IGNITE-18256 Add RecordView for C++ client

isapego commented on code in PR #1891:
URL: https://github.com/apache/ignite-3/pull/1891#discussion_r1156942453


##########
modules/platforms/cpp/ignite/client/table/record_view.h:
##########
@@ -475,4 +461,558 @@ class record_view<ignite_tuple> {
     std::shared_ptr<detail::table_impl> m_impl;
 };
 
+
+/**
+ * Record view interface provides methods to access table records.
+ */
+template<typename T>
+class record_view {
+    friend class table;
+
+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(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
+     */
+    void get_async(transaction *tx, const value_type &key, ignite_callback<std::optional<value_type>> callback) {
+        m_delegate.get_async(tx, convert_to_tuple(key), [callback = std::move(callback)] (auto res) {
+            callback(convert_result(std::move(res)));
+        });
+    }
+
+    /**
+     * 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]] 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 is 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.
+     */
+    void get_all_async(transaction *tx, std::vector<value_type> keys,
+        ignite_callback<std::vector<std::optional<value_type>>> callback) {
+        m_delegate.get_all_async(tx, values_to_tuples(std::move(keys)), [callback = std::move(callback)] (auto res) {
+            callback(convert_result(std::move(res)));
+        });
+    }
+
+    /**
+     * 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]] 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 existing 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.
+     */
+    void upsert_async(transaction *tx, const value_type &record, ignite_callback<void> callback) {
+        m_delegate.upsert_async(tx, convert_to_tuple(record), std::move(callback));
+    }
+
+    /**
+     * Inserts a record into the table if does not exist or replaces the existing 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.
+     */
+    void upsert(transaction *tx, const value_type &record) {
+        sync<void>([this, tx, &record](auto callback) { upsert_async(tx, record, std::move(callback)); });
+    }
+
+    /**
+     * Inserts multiple records into the table asynchronously, replacing
+     * existing.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param records Records to upsert.
+     * @param callback Callback that is called on operation completion.
+     */
+    void upsert_all_async(transaction *tx, std::vector<value_type> records, ignite_callback<void> callback) {
+        m_delegate.upsert_all_async(tx, values_to_tuples(std::move(records)), std::move(callback));
+    }
+
+    /**
+     * Inserts multiple records into the table, replacing existing.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param records Records to upsert.
+     */
+    void upsert_all(transaction *tx, std::vector<value_type> records) {
+        sync<void>([this, tx, records = std::move(records)](auto callback) mutable {
+            upsert_all_async(tx, std::move(records), std::move(callback));
+        });
+    }
+
+    /**
+     * Inserts a record into the table and returns previous record asynchronously.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to upsert.
+     * @param callback Callback. Called with a value which contains replaced
+     *   record or @c std::nullopt if it did not exist.
+     */
+    void get_and_upsert_async(
+        transaction *tx, const value_type &record, ignite_callback<std::optional<value_type>> callback) {
+        m_delegate.get_and_upsert_async(tx, convert_to_tuple(record), [callback = std::move(callback)] (auto res) {
+            callback(convert_result(std::move(res)));
+        });
+    }
+
+    /**
+     * Inserts a record into the table and returns previous record.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to upsert.
+     * @return A replaced record or @c std::nullopt if it did not exist.
+     */
+    [[nodiscard]] std::optional<value_type> get_and_upsert(transaction *tx, const value_type &record) {
+        return sync<std::optional<value_type>>(
+            [this, tx, &record](auto callback) { get_and_upsert_async(tx, record, std::move(callback)); });
+    }
+
+    /**
+     * Inserts a record into the table if it does not exist asynchronously.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to insert into the table.
+     * @param callback Callback. Called with a value indicating whether the
+     *   record was inserted. Equals @c false if a record with the same key
+     *   already exists.
+     */
+    void insert_async(transaction *tx, const value_type &record, ignite_callback<bool> callback) {
+        m_delegate.insert_async(tx, convert_to_tuple(record), std::move(callback));
+    }
+
+    /**
+     * Inserts a record into the table if does not exist.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to insert into the table.
+     */
+    bool insert(transaction *tx, const value_type &record) {
+        return sync<bool>([this, tx, &record](auto callback) { insert_async(tx, record, std::move(callback)); });
+    }
+
+    /**
+     * Inserts multiple records into the table asynchronously, skipping existing ones.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param records Records to insert.
+     * @param callback Callback that is called on operation completion. Called with
+     *   skipped records.
+     */
+    void insert_all_async(
+        transaction *tx, std::vector<value_type> records, ignite_callback<std::vector<value_type>> callback) {
+        m_delegate.insert_all_async(tx, values_to_tuples(std::move(records)),
+            [callback = std::move(callback)] (auto res) {
+                callback(convert_result(std::move(res)));
+            }
+        );
+    }
+
+    /**
+     * Inserts multiple records into the table, skipping existing ones.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param records Records to insert.
+     * @return Skipped records.
+     */
+    std::vector<value_type> insert_all(transaction *tx, std::vector<value_type> records) {
+        return sync<std::vector<value_type>>([this, tx, records = std::move(records)](auto callback) mutable {
+            insert_all_async(tx, std::move(records), std::move(callback));
+        });
+    }
+
+    /**
+     * Asynchronously replaces a record with the same key columns if it exists,
+     * otherwise does nothing.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to insert into the table.
+     * @param callback Callback. Called with a value indicating whether a record
+     *   with the specified key was replaced.
+     */
+    void replace_async(transaction *tx, const value_type &record, ignite_callback<bool> callback) {
+        m_delegate.replace_async(tx, convert_to_tuple(record), std::move(callback));
+    }
+
+    /**
+     * Replaces a record with the same key columns if it exists, otherwise does
+     * nothing.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to insert into the table.
+     * @return A value indicating whether a record with the specified key was
+     *   replaced.
+     */
+    bool replace(transaction *tx, const value_type &record) {
+        return sync<bool>([this, tx, &record](auto callback) { replace_async(tx, record, std::move(callback)); });
+    }
+
+    /**
+     * Asynchronously replaces a record with a new one only if all existing
+     * columns have the same values as the specified @c record.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record Current value of the record to be replaced.
+     * @param new_record A record to replace it with.
+     * @param callback Callback. Called with a value indicating whether a
+     *   specified record was replaced.
+     */
+    void replace_async(
+        transaction *tx, const value_type &record, const value_type &new_record, ignite_callback<bool> callback) {
+        m_delegate.replace_async(tx, convert_to_tuple(record), convert_to_tuple(new_record), std::move(callback));
+    }
+
+    /**
+     * Replaces a record with a new one only if all existing columns have
+     * the same values as the specified @c record.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record Current value of the record to be replaced.
+     * @param new_record A record to replace it with.
+     * @return A value indicating whether a specified record was replaced.
+     */
+    bool replace(transaction *tx, const value_type &record, const value_type &new_record) {
+        return sync<bool>([this, tx, &record, &new_record] (auto callback) {
+            replace_async(tx, record, new_record, std::move(callback));
+        });
+    }
+
+    /**
+     * Asynchronously replaces a record with the same key columns if it exists
+     * returning previous record value.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to insert.
+     * @param callback Callback. Called with a previous value for the given key,
+     *   or @c std::nullopt if it did not exist.
+     */
+    void get_and_replace_async(
+        transaction *tx, const value_type &record, ignite_callback<std::optional<value_type>> callback) {
+        m_delegate.get_and_replace_async(tx, convert_to_tuple(record), [callback = std::move(callback)] (auto res) {
+            callback(convert_result(std::move(res)));
+        });
+    }
+
+    /**
+     * Replaces a record with the same key columns if it exists returning
+     * previous record value.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param record A record to insert.
+     * @param callback A previous value for the given key, or @c std::nullopt if
+     *   it did not exist.
+     */
+    [[nodiscard]] std::optional<value_type> get_and_replace(transaction *tx, const value_type &record) {
+        return sync<std::optional<value_type>>(
+            [this, tx, &record](auto callback) { get_and_replace_async(tx, record, std::move(callback)); });
+    }
+
+    /**
+     * Deletes a record with the specified key asynchronously.
+     *
+     * @param tx Optional transaction. If nullptr implicit transaction for this
+     *   single operation is used.
+     * @param key A record with key columns set..

Review Comment:
   Fixed.



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