You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@arrow.apache.org by GitBox <gi...@apache.org> on 2022/08/08 16:33:03 UTC

[GitHub] [arrow-adbc] lidavidm opened a new issue, #55: [Format] Minor gaps with existing APIs

lidavidm opened a new issue, #55:
URL: https://github.com/apache/arrow-adbc/issues/55

   - A way to get the row count of a result set, when known (== DBAPI `Cursor.rowcount`)
   - A way to get the name of the 'current' catalog (== Ibis `Backend.current_database`)


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

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


[GitHub] [arrow-adbc] lidavidm commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
lidavidm commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1211235028

   Punting on paramstyle and last inserted ID, but adding row count and current catalog:
   
   ```diff
   commit 50b2e40d727c0a51029d7f5506c0696b3a19a3b9
   Author: David Li <li...@gmail.com>
   Date:   Wed Aug 10 16:29:27 2022 -0400
   
       [Format][C][Java] Add methods to get row count, current catalog
   
   diff --git a/adbc.h b/adbc.h
   index e7d9d51..5535193 100644
   --- a/adbc.h
   +++ b/adbc.h
   @@ -248,6 +248,18 @@ struct ADBC_EXPORT AdbcError {
      void (*release)(struct AdbcError* error);
    };
    
   +/// \brief A driver-allocated blob.
   +struct ADBC_EXPORT AdbcBlob {
   +  char* value;
   +  size_t length;
   +
   +  /// \brief Release the contained error.
   +  ///
   +  /// Unlike other structures, this is an embedded callback to make it
   +  /// easier for the driver manager and driver to cooperate.
   +  void (*release)(struct AdbcError* error);
   +};
   +
    /// }@
    
    /// \brief Canonical option value for enabling an option.
   @@ -342,6 +354,21 @@ ADBC_EXPORT
    AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection,
                                         struct AdbcError* error);
    
   +/// \defgroup adbc-connection-attributes Connection Attributes
   +/// Functions for retrieving metadata about the connection.
   +///
   +/// @{
   +
   +/// \brief Get the value of a string attribute.
   +ADBC_EXPORT
   +AdbcStatusCode AdbcConnectionGetAttr(struct AdbcConnection* connection, const char* key,
   +                                     struct AdbcBlob* blob, struct AdbcError* error);
   +
   +/// \brief The name of the attribute for the current catalog (database).
   +#define ADBC_CONNECTION_ATTR_CURRENT_CATALOG "adbc.connection.current_catalog"
   +
   +/// }@
   +
    /// \defgroup adbc-connection-metadata Metadata
    /// Functions for retrieving metadata about the database.
    ///
   @@ -746,6 +773,26 @@ AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement* statement,
                                           struct ArrowArrayStream* values,
                                           struct AdbcError* error);
    
   +/// \brief Get the number of rows affected by the most recent query.
   +///
   +/// Not supported by all drivers. If the query returns a result set,
   +/// this returns (if possible) the number of rows in the result
   +/// set. Else, this returns (if possible) the number of rows
   +/// inserted/updated.
   +///
   +/// This method can be called only after AdbcStatementExecute.  It may
   +/// not be called if any of the partitioning methods have been called
   +/// (see below).
   +///
   +/// \param[in] statement The statement.
   +/// \param[out] row_count The count of rows affected, or -1 if not
   +///   known.
   +/// \param[out] error An optional location to return an error message
   +///   if necessary.
   +ADBC_EXPORT
   +AdbcStatusCode AdbcStatementGetRowCount(struct AdbcStatement* statement,
   +                                        int64_t* row_count, struct AdbcError* error);
   +
    /// \brief Read the result of a statement.
    ///
    /// This method can be called only once per execution of the
   @@ -882,6 +929,8 @@ struct ADBC_EXPORT AdbcDriver {
      AdbcStatusCode (*DatabaseInit)(struct AdbcDatabase*, struct AdbcError*);
      AdbcStatusCode (*DatabaseRelease)(struct AdbcDatabase*, struct AdbcError*);
    
   +  AdbcStatusCode (*AdbcConnectionGetAttr)(struct AdbcConnection*, const char*,
   +                                          struct AdbcBlob*, struct AdbcError*);
      AdbcStatusCode (*ConnectionGetInfo)(struct AdbcConnection*, uint32_t*, size_t,
                                          struct AdbcStatement*, struct AdbcError*);
      AdbcStatusCode (*ConnectionNew)(struct AdbcConnection*, struct AdbcError*);
   @@ -924,6 +973,8 @@ struct ADBC_EXPORT AdbcDriver {
                                                      struct AdbcError*);
      AdbcStatusCode (*StatementGetPartitionDesc)(struct AdbcStatement*, uint8_t*,
                                                  struct AdbcError*);
   +  AdbcStatusCode (*StatementGetRowCount)(struct AdbcStatement*, int64_t*,
   +                                         struct AdbcError*);
      AdbcStatusCode (*StatementSetOption)(struct AdbcStatement*, const char*, const char*,
                                           struct AdbcError*);
      AdbcStatusCode (*StatementSetSqlQuery)(struct AdbcStatement*, const char*,
   diff --git a/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcConnection.java b/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcConnection.java
   index 3e3fe6f..cbea438 100644
   --- a/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcConnection.java
   +++ b/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcConnection.java
   @@ -54,8 +54,53 @@ public interface AdbcConnection extends AutoCloseable {
            "Connection does not support deserializePartitionDescriptor(ByteBuffer)");
      }
    
   +  /**
   +   * Get the name of the current catalog of this connection.
   +   *
   +   * @return The name, or <tt>null</tt> if not applicable.
   +   */
   +  String getCatalog() throws AdbcException;
   +
   +  /**
   +   * Get metadata about the database/driver.
   +   *
   +   * <p>The result is an Arrow dataset with the following schema:
   +   *
   +   * <table border="1">
   +   *   <tr><th>Field Name</th>                <th>Field Type</th>             </tr>
   +   *   <tr><td>info_name</td>                 <td>uint32 not null</td>        </tr>
   +   *   <tr><td>info_value</td>                <td>INFO_SCHEMA</td>            </tr>
   +   * </table>
   +   *
   +   * INFO_SCHEMA is a dense union with members:
   +   *
   +   * <table border="1">
   +   * <tr><th>Field Name (Type Code)</th>      <th>Field Type</th>             </tr>
   +   * <tr><td>string_value (0)</td>            <td>utf8</td>                   </tr>
   +   * <tr><td>bool_value (1)</td>              <td>bool</td>                   </tr>
   +   * <tr><td>int64_value (2)</td>             <td>int64</td>                  </tr>
   +   * <tr><td>int32_bitmask (3)</td>           <td>int32</td>                  </tr>
   +   * <tr><td>string_list (4)</td>             <td>list[utf8]</td>             </tr>
   +   * <tr><td>int32_to_int32_list_map (5)</td> <td>map[int32, list[int32]]</td></tr>
   +   * </table>
   +   *
   +   * Each metadatum is identified by an integer code. The recognized codes are defined as constants.
   +   * Codes [0, 10_000) are reserved for ADBC usage. Drivers/vendors will ignore requests for
   +   * unrecognized codes (the row will be omitted from the result).
   +   *
   +   * <p>\param[out] statement \param[out] error Error details, if an error occurs.
   +   *
   +   * @param infoCodes A list of metadata codes to fetch.
   +   * @return The result set. {@link AdbcStatement#getArrowReader()} can be called immediately; do
   +   *     not call {@link AdbcStatement#execute()}.
   +   */
      AdbcStatement getInfo(int[] infoCodes) throws AdbcException;
    
   +  /**
   +   * A convenience to get metadata with the type-safe enum.
   +   *
   +   * @see #getInfo(int[])
   +   */
      default AdbcStatement getInfo(AdbcInfoCode[] infoCodes) throws AdbcException {
        int[] codes = new int[infoCodes.length];
        for (int i = 0; i < infoCodes.length; i++) {
   @@ -64,6 +109,11 @@ public interface AdbcConnection extends AutoCloseable {
        return getInfo(codes);
      }
    
   +  /**
   +   * A convenience to get all known metadata.
   +   *
   +   * @see #getInfo(int[])
   +   */
      default AdbcStatement getInfo() throws AdbcException {
        return getInfo((int[]) null);
      }
   diff --git a/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcStatement.java b/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcStatement.java
   index b621587..eb7df7d 100644
   --- a/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcStatement.java
   +++ b/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcStatement.java
   @@ -65,10 +65,29 @@ public interface AdbcStatement extends AutoCloseable {
        return getArrowReader();
      }
    
   +  /**
   +   * Get the number of rows affected by the most recent query.
   +   *
   +   * <p>Not supported by all drivers. If the query returns a result set, this returns (if possible)
   +   * the number of rows in the result set. Else, this returns (if possible) the number of rows
   +   * inserted/updated.
   +   *
   +   * <p>This method should only be called after {@link #execute()}. It may not be called if any of
   +   * the partitioning methods have been called.
   +   *
   +   * @return The count of rows affected, or -1 if not known.
   +   * @throws AdbcException with {@link AdbcStatusCode#INVALID_STATE} if the statement has not been
   +   *     executed.
   +   */
   +  default long getRowCount() throws AdbcException {
   +    return -1;
   +  }
   +
      /**
       * Get the result of executing a query.
       *
   -   * <p>Must be called after {@link #execute()}.
   +   * <p>This method should only be called after {@link #execute()}. It may not be called if any of
   +   * the partitioning methods have been called.
       *
       * @throws AdbcException with {@link AdbcStatusCode#INVALID_STATE} if the statement has not been
       *     executed.
   ```


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

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


[GitHub] [arrow-adbc] lidavidm closed issue #55: [Format] Minor gaps with existing APIs

Posted by "lidavidm (via GitHub)" <gi...@apache.org>.
lidavidm closed issue #55: [Format] Minor gaps with existing APIs
URL: https://github.com/apache/arrow-adbc/issues/55


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

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


[GitHub] [arrow-adbc] lidavidm commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
lidavidm commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1211236715

   Returning strings from a C API is a bit annoying and I'm not sure whether this is preferable, or if we want to go with an ODBC-style API (pass a caller-allocated buffer and length and have the caller grow the buffer if too small)


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

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


[GitHub] [arrow-adbc] zeroshade commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
zeroshade commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1210860838

   Two more gaps to add:
   
   * Retrieve the last inserted id for inserts into an auto-increment table
   * Retrieve the number of rows affected by the last query (number inserted / number updated / etc.)


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

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


[GitHub] [arrow-adbc] lidavidm commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
lidavidm commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1211184131

   So looking into it
   - rowcount is easy to bind, but hard to support (lots of things don't support it or only support it for inserts) - that's OK. Flight SQL only exposes it for updates.
   - last inserted ID is harder to bind; JDBC requires you to specify that you want to get the information up front, while things like SQLite make it hard to get a reliable value. Flight SQL doesn't expose this.
   - paramstyle can just go in GetInfo but I think I'd rather punt on that sort of thing for now
   - current catalog (and current schema) are easily accessible from the connection JDBC. I don't immediately see how to get this for ODBC. Flight SQL doesn't really expose this.


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

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


[GitHub] [arrow-adbc] lidavidm commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
lidavidm commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1212119635

   > Is the idea that `RowCount` would do double duty as the number of rows in a result set OR the number of rows affected by an update/insert?
   
   Yeah, I don't see a reason to have two separate methods.
   
   > 
   > Given the lack of reliable support, i'm fine with leaving the LastInsertedID out for now.
   
   Also, I would argue these sorts of use cases are mostly out of scope, though that's mostly my assumption.
   
   > 
   > > Returning strings from a C API is a bit annoying....
   > 
   > Yea... I personally prefer the ODBC-style of passing a caller allocated buffer and length. But this works best when the caller can have a semblance or idea of how big of a buffer to allocate in the first place, so we'd have to either come up with a standard or some function to retrieve expected lengths of particular attributes so the caller can know how much to allocate, it might be simpler to go with the `AdbcBlob` approach you have here?
   
   It's also a little more consistent with AdbcError and the C Data Interface. But yeah, I don't think either is clearly better than the other?
   


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

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


[GitHub] [arrow-adbc] zeroshade commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
zeroshade commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1212115628

   Is the idea that `RowCount` would do double duty as the number of rows in a result set OR the number of rows affected by an update/insert?
   
   Given the lack of reliable support, i'm fine with leaving the LastInsertedID out for now.
   
   > Returning strings from a C API is a bit annoying....
   Yea... I personally prefer the ODBC-style of passing a caller allocated buffer and length. But this works best when the caller can have a semblance or idea of how big of a buffer to allocate in the first place, so we'd have to either come up with a standard or some function to retrieve expected lengths of particular attributes so the caller can know how much to allocate, it might be simpler to go with the `AdbcBlob` approach you have here?


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

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


[GitHub] [arrow-adbc] zeroshade commented on issue #55: [Format] Minor gaps with existing APIs

Posted by GitBox <gi...@apache.org>.
zeroshade commented on issue #55:
URL: https://github.com/apache/arrow-adbc/issues/55#issuecomment-1212124555

   Sounds good to me
   
   > Also, I would argue these sorts of use cases are mostly out of scope, though that's mostly my assumption.
   
   I agree, seems fine for that to be out of scope for the first pass here.
   
   > It's also a little more consistent with AdbcError and the C Data Interface. But yeah, I don't think either is clearly better than the other?
   
   Agreed. I vote we stay with consistency.


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

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