You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by li...@apache.org on 2023/06/27 17:29:58 UTC

[arrow-adbc] branch spec-1.1.0 updated: feat(format): add additional features to 1.1.0 spec (#765)

This is an automated email from the ASF dual-hosted git repository.

lidavidm pushed a commit to branch spec-1.1.0
in repository https://gitbox.apache.org/repos/asf/arrow-adbc.git


The following commit(s) were added to refs/heads/spec-1.1.0 by this push:
     new 9c9975ac feat(format): add additional features to 1.1.0 spec (#765)
9c9975ac is described below

commit 9c9975aced8ace2208ac08681d86242adc34f75a
Author: David Li <li...@gmail.com>
AuthorDate: Tue Jun 27 13:29:53 2023 -0400

    feat(format): add additional features to 1.1.0 spec (#765)
    
    Intends to tackle:
    
    - https://github.com/apache/arrow-adbc/issues/621
    - https://github.com/apache/arrow-adbc/issues/685
    - https://github.com/apache/arrow-adbc/issues/736
    - https://github.com/apache/arrow-adbc/issues/755
---
 adbc.h                                             |  449 ++++++++-
 c/driver_manager/adbc_driver_manager.cc            |   64 +-
 go/adbc/adbc.go                                    |  146 ++-
 go/adbc/drivermgr/adbc.h                           | 1008 +++++++++++++++++++-
 go/adbc/drivermgr/adbc_driver_manager.cc           |  473 ++++++++-
 .../org/apache/arrow/adbc/core/AdbcConnection.java |  103 +-
 .../org/apache/arrow/adbc/core/AdbcException.java  |   39 +-
 .../org/apache/arrow/adbc/core/AdbcStatement.java  |    2 +
 .../apache/arrow/adbc/core/StandardStatistics.java |   81 ++
 9 files changed, 2296 insertions(+), 69 deletions(-)

diff --git a/adbc.h b/adbc.h
index 5118f830..3e2d0f92 100644
--- a/adbc.h
+++ b/adbc.h
@@ -326,6 +326,33 @@ struct ADBC_EXPORT AdbcError {
 /// \since ADBC API revision 1.1.0
 /// \addtogroup adbc-1.1.0
 #define ADBC_OPTION_PASSWORD "password"
+/// \brief Canonical option name for error details.
+///
+/// Should be used as the expected option name to retrieve error
+/// details from the driver.  This allows drivers to return custom,
+/// structured error information (for example, JSON or Protocol
+/// Buffers) that can be optionally parsed by clients, beyond the
+/// standard AdbcError fields, without having to encode it in the
+/// error message.  The encoding of the data is driver-defined.
+///
+/// This can be called immediately after any API call that returns an
+/// error.  Additionally, if an ArrowArrayStream returned from an
+/// AdbcConnection or an AdbcStatement returns an error, this can be
+/// immediately called from the associated AdbcConnection or
+/// AdbcStatement to get further error details (if available).  Making
+/// other API calls with that connection or statement may clear this
+/// error value.
+///
+/// Drivers may provide multiple error details.  Each call to
+/// GetOptionBytes will return the next error detail.  The driver
+/// should return ADBC_STATUS_NOT_FOUND if there are no (more) error
+/// details.
+///
+/// The type is uint8_t*.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_OPTION_ERROR_DETAILS "error_details"
 
 /// \brief The database vendor/product name (e.g. the server name).
 ///   (type: utf8).
@@ -386,6 +413,54 @@ struct ADBC_EXPORT AdbcError {
 /// \see AdbcConnectionGetObjects
 #define ADBC_OBJECT_DEPTH_COLUMNS ADBC_OBJECT_DEPTH_ALL
 
+/// \defgroup adbc-table-statistics ADBC Statistic Types
+/// Standard statistic names for AdbcConnectionGetStatistics.
+/// @{
+
+/// \brief The dictionary-encoded name of the average byte width statistic.
+#define ADBC_STATISTIC_AVERAGE_BYTE_WIDTH_KEY 0
+/// \brief The average byte width statistic.  The average size in bytes of a
+///   row in the column.  Value type is float64.
+///
+/// For example, this is roughly the average length of a string for a string
+/// column.
+#define ADBC_STATISTIC_AVERAGE_BYTE_WIDTH_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the distinct value count statistic.
+#define ADBC_STATISTIC_DISTINCT_COUNT_KEY 1
+/// \brief The distinct value count (NDV) statistic.  The number of distinct
+///   values in the column.  Value type is int64 (when not approximate) or
+///   float64 (when approximate).
+#define ADBC_STATISTIC_DISTINCT_COUNT_NAME "adbc.statistic.distinct_count"
+/// \brief The dictionary-encoded name of the max byte width statistic.
+#define ADBC_STATISTIC_MAX_BYTE_WIDTH_KEY 2
+/// \brief The max byte width statistic.  The maximum size in bytes of a row
+///   in the column.  Value type is int64 (when not approximate) or float64
+///   (when approximate).
+///
+/// For example, this is the maximum length of a string for a string column.
+#define ADBC_STATISTIC_MAX_BYTE_WIDTH_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the max value statistic.
+#define ADBC_STATISTIC_MAX_VALUE_KEY 3
+/// \brief The max value statistic.  Value type is column-dependent.
+#define ADBC_STATISTIC_MAX_VALUE_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the min value statistic.
+#define ADBC_STATISTIC_MIN_VALUE_KEY 4
+/// \brief The min value statistic.  Value type is column-dependent.
+#define ADBC_STATISTIC_MIN_VALUE_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the null count statistic.
+#define ADBC_STATISTIC_NULL_COUNT_KEY 5
+/// \brief The null count statistic.  The number of values that are null in
+///   the column.  Value type is int64 (when not approximate) or float64
+///   (when approximate).
+#define ADBC_STATISTIC_NULL_COUNT_NAME "adbc.statistic.null_count"
+/// \brief The dictionary-encoded name of the row count statistic.
+#define ADBC_STATISTIC_ROW_COUNT_KEY 6
+/// \brief The row count statistic.  The number of rows in the column or
+///   table.  Value type is int64 (when not approximate) or float64 (when
+///   approximate).
+#define ADBC_STATISTIC_ROW_COUNT_NAME "adbc.statistic.row_count"
+/// @}
+
 /// \brief The name of the canonical option for whether autocommit is
 ///   enabled.
 ///
@@ -809,23 +884,38 @@ struct ADBC_EXPORT AdbcDriver {
   ///
   /// @{
 
-  AdbcStatusCode (*DatabaseGetOption)(struct AdbcDatabase*, const char*, const char**,
+  AdbcStatusCode (*DatabaseGetOption)(struct AdbcDatabase*, const char*, char*, size_t*,
                                       struct AdbcError*);
+  AdbcStatusCode (*DatabaseGetOptionBytes)(struct AdbcDatabase*, const char*, uint8_t*,
+                                           size_t*, struct AdbcError*);
   AdbcStatusCode (*DatabaseGetOptionInt)(struct AdbcDatabase*, const char*, int64_t*,
                                          struct AdbcError*);
   AdbcStatusCode (*DatabaseGetOptionDouble)(struct AdbcDatabase*, const char*, double*,
                                             struct AdbcError*);
+  AdbcStatusCode (*DatabaseSetOptionBytes)(struct AdbcDatabase*, const char*,
+                                           const uint8_t*, size_t, struct AdbcError*);
   AdbcStatusCode (*DatabaseSetOptionInt)(struct AdbcDatabase*, const char*, int64_t,
                                          struct AdbcError*);
   AdbcStatusCode (*DatabaseSetOptionDouble)(struct AdbcDatabase*, const char*, double,
                                             struct AdbcError*);
 
-  AdbcStatusCode (*ConnectionGetOption)(struct AdbcConnection*, const char*, const char**,
-                                        struct AdbcError*);
+  AdbcStatusCode (*ConnectionCancel)(struct AdbcConnection*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetOption)(struct AdbcConnection*, const char*, char*,
+                                        size_t*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetOptionBytes)(struct AdbcDatabase*, const char*, uint8_t*,
+                                             size_t*, struct AdbcError*);
   AdbcStatusCode (*ConnectionGetOptionInt)(struct AdbcConnection*, const char*, int64_t*,
                                            struct AdbcError*);
   AdbcStatusCode (*ConnectionGetOptionDouble)(struct AdbcConnection*, const char*,
                                               double*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetStatistics)(struct AdbcConnection*, const char*,
+                                            const char*, const char*, char,
+                                            struct ArrowArrayStream*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetStatisticNames)(struct AdbcConnection*,
+                                                struct ArrowArrayStream*,
+                                                struct AdbcError*);
+  AdbcStatusCode (*ConnectionSetOptionBytes)(struct AdbcDatabase*, const char*,
+                                             const uint8_t*, size_t, struct AdbcError*);
   AdbcStatusCode (*ConnectionSetOptionInt)(struct AdbcConnection*, const char*, int64_t,
                                            struct AdbcError*);
   AdbcStatusCode (*ConnectionSetOptionDouble)(struct AdbcConnection*, const char*, double,
@@ -834,12 +924,16 @@ struct ADBC_EXPORT AdbcDriver {
   AdbcStatusCode (*StatementCancel)(struct AdbcStatement*, struct AdbcError*);
   AdbcStatusCode (*StatementExecuteSchema)(struct AdbcStatement*, struct ArrowSchema*,
                                            struct AdbcError*);
-  AdbcStatusCode (*StatementGetOption)(struct AdbcStatement*, const char*, const char**,
+  AdbcStatusCode (*StatementGetOption)(struct AdbcStatement*, const char*, char*, size_t*,
                                        struct AdbcError*);
+  AdbcStatusCode (*StatementGetOptionBytes)(struct AdbcDatabase*, const char*, uint8_t*,
+                                            size_t*, struct AdbcError*);
   AdbcStatusCode (*StatementGetOptionInt)(struct AdbcStatement*, const char*, int64_t*,
                                           struct AdbcError*);
   AdbcStatusCode (*StatementGetOptionDouble)(struct AdbcStatement*, const char*, double*,
                                              struct AdbcError*);
+  AdbcStatusCode (*StatementSetOptionBytes)(struct AdbcDatabase*, const char*,
+                                            const uint8_t*, size_t, struct AdbcError*);
   AdbcStatusCode (*StatementSetOptionInt)(struct AdbcStatement*, const char*, int64_t,
                                           struct AdbcError*);
   AdbcStatusCode (*StatementSetOptionDouble)(struct AdbcStatement*, const char*, double,
@@ -883,10 +977,23 @@ AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError*
 
 /// \brief Get a string option of the database.
 ///
-/// This must always be thread-safe (other operations are not).
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call GetOption
+/// concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value (including the null terminator) to buffer and set
+/// length to the size of the actual value.  If the buffer is too
+/// small, no data will be written and length will be set to the
+/// required length.
 ///
-/// The returned option value is only valid until the next call to
-/// GetOption or Release.
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
 ///
 /// For standard options, drivers must always support getting the
 /// option value (if they support getting option values at all) via
@@ -901,11 +1008,53 @@ AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError*
 /// \param[in] database The database.
 /// \param[in] key The option to get.
 /// \param[out] value The option value.
+/// \param[in,out] length The length of value.
 /// \param[out] error An optional location to return an error
 ///   message if necessary.
 /// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
 AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase* database, const char* key,
-                                     const char** value, struct AdbcError* error);
+                                     char* value, size_t* length,
+                                     struct AdbcError* error);
+
+/// \brief Get a bytestring option of the database.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call
+/// GetOptionBytes concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value to buffer and set length to the size of the
+/// actual value.  If the buffer is too small, no data will be written
+/// and length will be set to the required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcDatabaseGetOptionBytes(struct AdbcDatabase* database, const char* key,
+                                          uint8_t* value, size_t* length,
+                                          struct AdbcError* error);
 
 /// \brief Get an integer option of the database.
 ///
@@ -968,6 +1117,22 @@ ADBC_EXPORT
 AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase* database, const char* key,
                                      const char* value, struct AdbcError* error);
 
+/// \brief Set a bytestring option on a database.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[in] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcDatabaseSetOptionBytes(struct AdbcDatabase* database, const char* key,
+                                          const uint8_t* value, size_t length,
+                                          struct AdbcError* error);
+
 /// \brief Set an integer option on a database.
 ///
 /// \since ADBC API revision 1.1.0
@@ -1042,6 +1207,22 @@ ADBC_EXPORT
 AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection* connection, const char* key,
                                        const char* value, struct AdbcError* error);
 
+/// \brief Set a bytestring option on a connection.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The connection.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[in] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionSetOptionBytes(struct AdbcConnection* connection,
+                                            const char* key, const uint8_t* value,
+                                            size_t length, struct AdbcError* error);
+
 /// \brief Set an integer option.
 ///
 /// Options may be set before AdbcConnectionInit.  Some drivers may
@@ -1095,6 +1276,29 @@ ADBC_EXPORT
 AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection,
                                      struct AdbcError* error);
 
+/// \brief Cancel the in-progress operation on a connection.
+///
+/// This can be called during AdbcConnectionGetObjects (or similar),
+/// or while consuming an ArrowArrayStream returned from such.
+/// Calling this function should make the other functions return
+/// ADBC_STATUS_CANCELLED (from ADBC functions) or ECANCELED (from
+/// methods of ArrowArrayStream).
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+///
+/// \param[in] connection The connection to cancel.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+///
+/// \return ADBC_STATUS_INVALID_STATE if there is no operation to cancel.
+/// \return ADBC_STATUS_UNKNOWN if the operation could not be cancelled.
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection* connection,
+                                    struct AdbcError* error);
+
 /// \defgroup adbc-connection-metadata Metadata
 /// Functions for retrieving metadata about the database.
 ///
@@ -1275,10 +1479,55 @@ AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int d
 
 /// \brief Get a string option of the connection.
 ///
-/// This must always be thread-safe (other operations are not).
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call GetOption
+/// concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value (including the null terminator) to buffer and set
+/// length to the size of the actual value.  If the buffer is too
+/// small, no data will be written and length will be set to the
+/// required length.
+///
+/// In other words:
 ///
-/// The returned option value is only valid until the next call to
-/// GetOption or Release.
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The length of value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const char* key,
+                                       char* value, size_t* length,
+                                       struct AdbcError* error);
+
+/// \brief Get a bytestring option of the connection.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call
+/// GetOptionBytes concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value to buffer and set length to the size of the
+/// actual value.  If the buffer is too small, no data will be written
+/// and length will be set to the required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
 ///
 /// For standard options, drivers must always support getting the
 /// option value (if they support getting option values at all) via
@@ -1290,14 +1539,16 @@ AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int d
 ///
 /// \since ADBC API revision 1.1.0
 /// \addtogroup adbc-1.1.0
-/// \param[in] connection The database connection.
+/// \param[in] connection The connection.
 /// \param[in] key The option to get.
 /// \param[out] value The option value.
+/// \param[in,out] length The option value length.
 /// \param[out] error An optional location to return an error
 ///   message if necessary.
 /// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
-AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const char* key,
-                                       const char** value, struct AdbcError* error);
+AdbcStatusCode AdbcConnectionGetOptionBytes(struct AdbcConnection* connection,
+                                            const char* key, uint8_t* value,
+                                            size_t* length, struct AdbcError* error);
 
 /// \brief Get an integer option of the connection.
 ///
@@ -1347,6 +1598,94 @@ AdbcStatusCode AdbcConnectionGetOptionDouble(struct AdbcConnection* connection,
                                              const char* key, double* value,
                                              struct AdbcError* error);
 
+/// \brief Get statistics about the data distribution of table(s).
+///
+/// The result is an Arrow dataset with the following schema:
+///
+/// | Field Name               | Field Type                       |
+/// |--------------------------|----------------------------------|
+/// | catalog_name             | utf8                             |
+/// | catalog_db_schemas       | list<DB_SCHEMA_SCHEMA>           |
+///
+/// DB_SCHEMA_SCHEMA is a Struct with fields:
+///
+/// | Field Name               | Field Type                       |
+/// |--------------------------|----------------------------------|
+/// | db_schema_name           | utf8                             |
+/// | db_schema_functions      | list<STATISTICS_SCHEMA>          |
+///
+/// STATISTICS_SCHEMA is a Struct with fields:
+///
+/// | Field Name               | Field Type                       | Comments |
+/// |--------------------------|----------------------------------| -------- |
+/// | table_name               | utf8 not null                    |          |
+/// | column_name              | utf8                             | (1)      |
+/// | statistic_key            | int16 not null                   | (2)      |
+/// | statistic_value          | VALUE_SCHEMA not null            |          |
+/// | statistic_is_approximate | bool not null                    | (3)      |
+///
+/// 1. If null, then the statistic applies to the entire table.
+/// 2. A dictionary-encoded statistic name (although we do not use the Arrow
+///    dictionary type). Values in [0, 1024) are reserved for ADBC.  Other
+///    values are for implementation-specific statistics.  For the definitions
+///    of predefined statistic types, see \ref adbc-table-statistics.  To get
+///    driver-specific statistic names, use AdbcConnectionGetStatisticNames.
+/// 3. If true, then the value is approximate or best-effort.
+///
+/// VALUE_SCHEMA is a dense union with members:
+///
+/// | Field Name               | Field Type                       |
+/// |--------------------------|----------------------------------|
+/// | int64                    | int64                            |
+/// | uint64                   | uint64                           |
+/// | float64                  | float64                          |
+/// | decimal256               | decimal256                       |
+/// | binary                   | binary                           |
+///
+/// This AdbcConnection must outlive the returned ArrowArrayStream.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] catalog The catalog (or nullptr).  May be a search
+///   pattern (see section documentation).
+/// \param[in] db_schema The database schema (or nullptr).  May be a
+///   search pattern (see section documentation).
+/// \param[in] table_name The table name (or nullptr).  May be a
+///   search pattern (see section documentation).
+/// \param[in] approximate If zero, request exact values of
+///   statistics, else allow for best-effort, approximate, or cached
+///   values.  The database may return approximate values regardless,
+///   as indicated in the result.  Requesting exact values may be
+///   expensive or unsupported.
+/// \param[out] out The result set.
+/// \param[out] error Error details, if an error occurs.
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionGetStatistics(struct AdbcConnection* connection,
+                                           const char* catalog, const char* db_schema,
+                                           const char* table_name, char approximate,
+                                           struct ArrowArrayStream* out,
+                                           struct AdbcError* error);
+
+/// \brief Get the names of statistics specific to this driver.
+///
+/// The result is an Arrow dataset with the following schema:
+///
+/// Field Name     | Field Type
+/// ---------------|----------------
+/// statistic_name | utf8 not null
+/// statistic_key  | int16 not null
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[out] out The result set.
+/// \param[out] error Error details, if an error occurs.
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionGetStatisticNames(struct AdbcConnection* connection,
+                                               struct ArrowArrayStream* out,
+                                               struct AdbcError* error);
+
 /// \brief Get the Arrow schema of a table.
 ///
 /// \param[in] connection The database connection.
@@ -1494,6 +1833,9 @@ AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement* statement,
 ///
 /// This invalidates any prior result sets.
 ///
+/// Depending on the driver, this may require first executing
+/// AdbcStatementPrepare.
+///
 /// \since ADBC API revision 1.1.0
 /// \addtogroup adbc-1.1.0
 ///
@@ -1615,10 +1957,23 @@ AdbcStatusCode AdbcStatementCancel(struct AdbcStatement* statement,
 
 /// \brief Get a string option of the statement.
 ///
-/// This must always be thread-safe (other operations are not).
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call GetOption
+/// concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value (including the null terminator) to buffer and set
+/// length to the size of the actual value.  If the buffer is too
+/// small, no data will be written and length will be set to the
+/// required length.
 ///
-/// The returned option value is only valid until the next call to
-/// GetOption or Release.
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
 ///
 /// For standard options, drivers must always support getting the
 /// option value (if they support getting option values at all) via
@@ -1633,11 +1988,53 @@ AdbcStatusCode AdbcStatementCancel(struct AdbcStatement* statement,
 /// \param[in] statement The statement.
 /// \param[in] key The option to get.
 /// \param[out] value The option value.
+/// \param[in,out] length The length of value.
 /// \param[out] error An optional location to return an error
 ///   message if necessary.
 /// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
 AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement* statement, const char* key,
-                                      const char** value, struct AdbcError* error);
+                                      char* value, size_t* length,
+                                      struct AdbcError* error);
+
+/// \brief Get a bytestring option of the statement.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call
+/// GetOptionBytes concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value to buffer and set length to the size of the
+/// actual value.  If the buffer is too small, no data will be written
+/// and length will be set to the required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcStatementGetOptionBytes(struct AdbcStatement* statement,
+                                           const char* key, uint8_t* value,
+                                           size_t* length, struct AdbcError* error);
 
 /// \brief Get an integer option of the statement.
 ///
@@ -1717,6 +2114,22 @@ ADBC_EXPORT
 AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement* statement, const char* key,
                                       const char* value, struct AdbcError* error);
 
+/// \brief Set a bytestring option on a statement.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[in] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcStatementSetOptionBytes(struct AdbcStatement* statement,
+                                           const char* key, const uint8_t* value,
+                                           size_t length, struct AdbcError* error);
+
 /// \brief Set an integer option on a statement.
 ///
 /// \since ADBC API revision 1.1.0
diff --git a/c/driver_manager/adbc_driver_manager.cc b/c/driver_manager/adbc_driver_manager.cc
index e0315d86..8486bf1a 100644
--- a/c/driver_manager/adbc_driver_manager.cc
+++ b/c/driver_manager/adbc_driver_manager.cc
@@ -131,7 +131,7 @@ static AdbcStatusCode ReleaseDriver(struct AdbcDriver* driver, struct AdbcError*
 // Default stubs
 
 AdbcStatusCode DatabaseGetOption(struct AdbcDatabase* database, const char* key,
-                                 const char** value, struct AdbcError* error) {
+                                 char* value, size_t* length, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
@@ -160,6 +160,11 @@ AdbcStatusCode DatabaseSetOptionDouble(struct AdbcDatabase* database, const char
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode ConnectionCancel(struct AdbcConnection* connection,
+                                struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode ConnectionCommit(struct AdbcConnection*, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
@@ -177,7 +182,7 @@ AdbcStatusCode ConnectionGetObjects(struct AdbcConnection*, int, const char*, co
 }
 
 AdbcStatusCode ConnectionGetOption(struct AdbcConnection* connection, const char* key,
-                                   const char** value, struct AdbcError* error) {
+                                   char* value, size_t* length, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
@@ -255,7 +260,7 @@ AdbcStatusCode StatementExecuteSchema(struct AdbcStatement* statement,
 }
 
 AdbcStatusCode StatementGetOption(struct AdbcStatement* statement, const char* key,
-                                  const char** value, struct AdbcError* error) {
+                                  char* value, size_t* length, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
@@ -333,22 +338,31 @@ AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError*
 }
 
 AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase* database, const char* key,
-                                     const char** value, struct AdbcError* error) {
+                                     char* value, size_t* length,
+                                     struct AdbcError* error) {
   if (database->private_driver) {
-    return database->private_driver->DatabaseGetOption(database, key, value, error);
+    return database->private_driver->DatabaseGetOption(database, key, value, length,
+                                                       error);
   }
   const auto* args = reinterpret_cast<const TempDatabase*>(database->private_data);
+  const std::string* result = nullptr;
   if (std::strcmp(key, "driver") == 0) {
-    *value = args->driver.c_str();
+    result = &args->driver;
   } else if (std::strcmp(key, "entrypoint") == 0) {
-    *value = args->entrypoint.c_str();
+    result = &args->entrypoint;
   } else {
     const auto it = args->options.find(key);
     if (it == args->options.end()) {
       return ADBC_STATUS_NOT_FOUND;
     }
-    *value = it->second.c_str();
+    result = &it->second;
   }
+
+  if (*length <= result->size() + 1) {
+    // Enough space
+    std::memcpy(value, result->c_str(), result->size() + 1);
+  }
+  *length = result->size() + 1;
   return ADBC_STATUS_OK;
 }
 
@@ -533,6 +547,14 @@ AdbcStatusCode AdbcDatabaseRelease(struct AdbcDatabase* database,
   return status;
 }
 
+AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection* connection,
+                                    struct AdbcError* error) {
+  if (!connection->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return connection->private_driver->ConnectionCancel(connection, error);
+}
+
 AdbcStatusCode AdbcConnectionCommit(struct AdbcConnection* connection,
                                     struct AdbcError* error) {
   if (!connection->private_driver) {
@@ -567,7 +589,8 @@ AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int d
 }
 
 AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const char* key,
-                                       const char** value, struct AdbcError* error) {
+                                       char* value, size_t* length,
+                                       struct AdbcError* error) {
   if (!connection->private_data) {
     SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first");
     return ADBC_STATUS_INVALID_STATE;
@@ -579,10 +602,14 @@ AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const
     if (it == args->options.end()) {
       return ADBC_STATUS_NOT_FOUND;
     }
-    *value = it->second.c_str();
+    if (*length >= it->second.size() + 1) {
+      std::memcpy(value, it->second.c_str(), it->second.size() + 1);
+    }
+    *length = it->second.size() + 1;
     return ADBC_STATUS_OK;
   }
-  return connection->private_driver->ConnectionGetOption(connection, key, value, error);
+  return connection->private_driver->ConnectionGetOption(connection, key, value, length,
+                                                         error);
 }
 
 AdbcStatusCode AdbcConnectionGetOptionInt(struct AdbcConnection* connection,
@@ -801,6 +828,14 @@ AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement* statement,
   return statement->private_driver->StatementBindStream(statement, stream, error);
 }
 
+AdbcStatusCode AdbcStatementCancel(struct AdbcStatement* statement,
+                                   struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementCancel(statement, error);
+}
+
 // XXX: cpplint gets confused here if declared as 'struct ArrowSchema* schema'
 AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement* statement,
                                               ArrowSchema* schema,
@@ -835,11 +870,13 @@ AdbcStatusCode AdbcStatementExecuteSchema(struct AdbcStatement* statement,
 }
 
 AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement* statement, const char* key,
-                                      const char** value, struct AdbcError* error) {
+                                      char* value, size_t* length,
+                                      struct AdbcError* error) {
   if (!statement->private_driver) {
     return ADBC_STATUS_INVALID_STATE;
   }
-  return statement->private_driver->StatementGetOption(statement, key, value, error);
+  return statement->private_driver->StatementGetOption(statement, key, value, length,
+                                                       error);
 }
 
 AdbcStatusCode AdbcStatementGetOptionInt(struct AdbcStatement* statement, const char* key,
@@ -1197,6 +1234,7 @@ AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int vers
     FILL_DEFAULT(driver, DatabaseSetOptionInt);
     FILL_DEFAULT(driver, DatabaseSetOptionDouble);
 
+    FILL_DEFAULT(driver, ConnectionCancel);
     FILL_DEFAULT(driver, ConnectionGetOption);
     FILL_DEFAULT(driver, ConnectionGetOptionInt);
     FILL_DEFAULT(driver, ConnectionGetOptionDouble);
diff --git a/go/adbc/adbc.go b/go/adbc/adbc.go
index bc03d428..99a4f81b 100644
--- a/go/adbc/adbc.go
+++ b/go/adbc/adbc.go
@@ -58,6 +58,13 @@ type Error struct {
 	// SqlState is a SQLSTATE error code, if provided, as defined
 	// by the SQL:2003 standard. If not set, it will be "\0\0\0\0\0"
 	SqlState [5]byte
+	// Details is an array of additional driver-specific binary error details.
+	//
+	// This allows drivers to return custom, structured error information (for
+	// example, JSON or Protocol Buffers) that can be optionally parsed by
+	// clients, beyond the standard Error fields, without having to encode it in
+	// the error message.  The encoding of the data is driver-defined.
+	Details [][]byte
 }
 
 func (e Error) Error() string {
@@ -190,6 +197,51 @@ const (
 	PropertyProgress = "adbc.statement.exec.progress"
 )
 
+// Standard statistic names and keys.
+const (
+	// The dictionary-encoded name of the average byte width statistic.
+	StatisticAverageByteWidthKey = 0
+	// The average byte width statistic.  The average size in bytes of a row in
+	// the column.  Value type is float64.
+	//
+	// For example, this is roughly the average length of a string for a string
+	// column.
+	StatisticAverageByteWidthName = "adbc.statistic.byte_width"
+	// The dictionary-encoded name of the distinct value count statistic.
+	StatisticDistinctCountKey = 1
+	// The distinct value count (NDV) statistic.  The number of distinct values in
+	// the column.  Value type is int64 (when not approximate) or float64 (when
+	// approximate).
+	StatisticDistinctCountName = "adbc.statistic.distinct_count"
+	// The dictionary-encoded name of the max byte width statistic.
+	StatisticMaxByteWidthKey = 2
+	// The max byte width statistic.  The maximum size in bytes of a row in the
+	// column.  Value type is int64 (when not approximate) or float64 (when
+	// approximate).
+	//
+	// For example, this is the maximum length of a string for a string column.
+	StatisticMaxByteWidthName = "adbc.statistic.byte_width"
+	// The dictionary-encoded name of the max value statistic.
+	StatisticMaxValueKey = 3
+	// The max value statistic.  Value type is column-dependent.
+	StatisticMaxValueName = "adbc.statistic.byte_width"
+	// The dictionary-encoded name of the min value statistic.
+	StatisticMinValueKey = 4
+	// The min value statistic.  Value type is column-dependent.
+	StatisticMinValueName = "adbc.statistic.byte_width"
+	// The dictionary-encoded name of the null count statistic.
+	StatisticNullCountKey = 5
+	// The null count statistic.  The number of values that are null in the
+	// column.  Value type is int64 (when not approximate) or float64 (when
+	// approximate).
+	StatisticNullCountName = "adbc.statistic.null_count"
+	// The dictionary-encoded name of the row count statistic.
+	StatisticRowCountKey = 6
+	// The row count statistic.  The number of rows in the column or table.  Value
+	// type is int64 (when not approximate) or float64 (when approximate).
+	StatisticRowCountName = "adbc.statistic.row_count"
+)
+
 // Driver is the entry point for the interface. It is similar to
 // database/sql.Driver taking a map of keys and values as options
 // to initialize a Connection to the database. Any common connection
@@ -569,22 +621,96 @@ type Statement interface {
 	ExecutePartitions(context.Context) (*arrow.Schema, Partitions, int64, error)
 }
 
-// StatementCancel is a Statement that also supports Cancel.
+// Cancellable is a Connection or Statement that also supports Cancel.
 //
 // Since ADBC API revision 1.1.0.
-type StatementCancel interface {
+type Cancellable interface {
 	// Cancel stops execution of an in-progress query.
 	//
-	// This can be called during ExecuteQuery (or similar), or while
-	// consuming a RecordReader returned from such.  Calling this
-	// function should make the other functions return an error with a
-	// StatusCancelled code.
+	// This can be called during ExecuteQuery, GetObjects, or other
+	// methods that produce result sets, or while consuming a
+	// RecordReader returned from such.  Calling this function should
+	// make the other functions return an error with a StatusCancelled
+	// code.
 	//
 	// This must always be thread-safe (other operations are not
 	// necessarily thread-safe).
 	Cancel() error
 }
 
+// ConnectionGetStatistics is a Connection that supports getting
+// statistics on data in the database.
+//
+// Since ADBC API revision 1.1.0.
+type ConnectionGetStatistics interface {
+	// GetStatistics gets statistics about the data distribution of table(s).
+	//
+	// The result is an Arrow dataset with the following schema:
+	//
+	//		Field Name               | Field Type
+	//		-------------------------|----------------------------------
+	//		catalog_name             | utf8
+	//		catalog_db_schemas       | list<DB_SCHEMA_SCHEMA>
+	//
+	// DB_SCHEMA_SCHEMA is a Struct with fields:
+	//
+	//		Field Name               | Field Type
+	//		-------------------------|----------------------------------
+	//		db_schema_name           | utf8
+	//		db_schema_functions      | list<STATISTICS_SCHEMA>
+	//
+	// STATISTICS_SCHEMA is a Struct with fields:
+	//
+	//		Field Name               | Field Type                       | Comments
+	//		-------------------------|----------------------------------| --------
+	//		table_name               | utf8 not null                    |
+	//		column_name              | utf8                             | (1)
+	//		statistic_key            | int16 not null                   | (2)
+	//		statistic_value          | VALUE_SCHEMA not null            |
+	//		statistic_is_approximate | bool not null                    | (3)
+	//
+	// 1. If null, then the statistic applies to the entire table.
+	// 2. A dictionary-encoded statistic name (although we do not use the Arrow
+	//    dictionary type). Values in [0, 1024) are reserved for ADBC.  Other
+	//    values are for implementation-specific statistics.  For the definitions
+	//    of predefined statistic types, see the Statistic constants.  To get
+	//    driver-specific statistic names, use AdbcConnectionGetStatisticNames.
+	// 3. If true, then the value is approximate or best-effort.
+	//
+	// VALUE_SCHEMA is a dense union with members:
+	//
+	//		Field Name               | Field Type
+	//		-------------------------|----------------------------------
+	//		int64                    | int64
+	//		uint64                   | uint64
+	//		float64                  | float64
+	//		decimal256               | decimal256
+	//		binary                   | binary
+	//
+	// For the parameters: If nil is passed, then that parameter will not
+	// be filtered by at all. If an empty string, then only objects without
+	// that property (ie: catalog or db schema) will be returned.
+	//
+	// All non-empty, non-nil strings should be a search pattern (as described
+	// earlier).
+	//
+	// approximate indicates whether to request exact values of statistics, or
+	// best-effort/cached values. Requesting exact values may be expensive or
+	// unsupported.
+	GetStatistics(ctx context.Context, catalog, dbSchema, tableName *string, approximate bool) (array.RecordReader, error)
+
+	// GetStatisticNames gets a list of custom statistic names defined by this driver.
+	//
+	// The result is an Arrow dataset with the following schema:
+	//
+	//		Field Name     | Field Type
+	//		---------------|----------------
+	//		statistic_name | utf8 not null
+	//		statistic_key  | int16 not null
+	//
+	GetStatisticNames() (array.RecordReader, error)
+}
+
 // StatementExecuteSchema is a Statement that also supports ExecuteSchema.
 //
 // Since ADBC API revision 1.1.0.
@@ -599,9 +725,11 @@ type StatementExecuteSchema interface {
 type GetSetOptions interface {
 	PostInitOptions
 
-	SetOption(key, value string) error
-	SetOptionInt(key, value int64) error
-	SetOptionDouble(key, value float64) error
+	SetOptionBytes(key string, value []byte) error
+	SetOptionInt(key string, value int64) error
+	SetOptionDouble(key string, value float64) error
+	GetOption(key, value string) (string, error)
+	GetOptionBytes(key string) ([]byte, error)
 	GetOptionInt(key string) (int64, error)
 	GetOptionDouble(key string) (float64, error)
 }
diff --git a/go/adbc/drivermgr/adbc.h b/go/adbc/drivermgr/adbc.h
index 154e8812..3e2d0f92 100644
--- a/go/adbc/drivermgr/adbc.h
+++ b/go/adbc/drivermgr/adbc.h
@@ -279,6 +279,14 @@ struct ADBC_EXPORT AdbcError {
 /// point to an AdbcDriver.
 #define ADBC_VERSION_1_0_0 1000000
 
+/// \brief ADBC revision 1.1.0.
+///
+/// When passed to an AdbcDriverInitFunc(), the driver parameter must
+/// point to an AdbcDriver.
+///
+/// \addtogroup adbc-1.1.0
+#define ADBC_VERSION_1_1_0 1001000
+
 /// \brief Canonical option value for enabling an option.
 ///
 /// For use as the value in SetOption calls.
@@ -288,6 +296,64 @@ struct ADBC_EXPORT AdbcError {
 /// For use as the value in SetOption calls.
 #define ADBC_OPTION_VALUE_DISABLED "false"
 
+/// \brief Canonical option name for URIs.
+///
+/// Should be used as the expected option name to specify a URI for
+/// any ADBC driver.
+///
+/// The type is char*.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_OPTION_URI "uri"
+/// \brief Canonical option name for usernames.
+///
+/// Should be used as the expected option name to specify a username
+/// to a driver for authentication.
+///
+/// The type is char*.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_OPTION_USERNAME "username"
+/// \brief Canonical option name for passwords.
+///
+/// Should be used as the expected option name to specify a password
+/// for authentication to a driver.
+///
+/// The type is char*.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_OPTION_PASSWORD "password"
+/// \brief Canonical option name for error details.
+///
+/// Should be used as the expected option name to retrieve error
+/// details from the driver.  This allows drivers to return custom,
+/// structured error information (for example, JSON or Protocol
+/// Buffers) that can be optionally parsed by clients, beyond the
+/// standard AdbcError fields, without having to encode it in the
+/// error message.  The encoding of the data is driver-defined.
+///
+/// This can be called immediately after any API call that returns an
+/// error.  Additionally, if an ArrowArrayStream returned from an
+/// AdbcConnection or an AdbcStatement returns an error, this can be
+/// immediately called from the associated AdbcConnection or
+/// AdbcStatement to get further error details (if available).  Making
+/// other API calls with that connection or statement may clear this
+/// error value.
+///
+/// Drivers may provide multiple error details.  Each call to
+/// GetOptionBytes will return the next error detail.  The driver
+/// should return ADBC_STATUS_NOT_FOUND if there are no (more) error
+/// details.
+///
+/// The type is uint8_t*.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_OPTION_ERROR_DETAILS "error_details"
+
 /// \brief The database vendor/product name (e.g. the server name).
 ///   (type: utf8).
 ///
@@ -315,6 +381,16 @@ struct ADBC_EXPORT AdbcError {
 ///
 /// \see AdbcConnectionGetInfo
 #define ADBC_INFO_DRIVER_ARROW_VERSION 102
+/// \brief The driver ADBC API version (type: int64).
+///
+/// The value should be one of the ADBC_VERSION constants.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \see AdbcConnectionGetInfo
+/// \see ADBC_VERSION_1_0_0
+/// \see ADBC_VERSION_1_1_0
+#define ADBC_INFO_DRIVER_ADBC_VERSION 103
 
 /// \brief Return metadata on catalogs, schemas, tables, and columns.
 ///
@@ -337,18 +413,119 @@ struct ADBC_EXPORT AdbcError {
 /// \see AdbcConnectionGetObjects
 #define ADBC_OBJECT_DEPTH_COLUMNS ADBC_OBJECT_DEPTH_ALL
 
+/// \defgroup adbc-table-statistics ADBC Statistic Types
+/// Standard statistic names for AdbcConnectionGetStatistics.
+/// @{
+
+/// \brief The dictionary-encoded name of the average byte width statistic.
+#define ADBC_STATISTIC_AVERAGE_BYTE_WIDTH_KEY 0
+/// \brief The average byte width statistic.  The average size in bytes of a
+///   row in the column.  Value type is float64.
+///
+/// For example, this is roughly the average length of a string for a string
+/// column.
+#define ADBC_STATISTIC_AVERAGE_BYTE_WIDTH_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the distinct value count statistic.
+#define ADBC_STATISTIC_DISTINCT_COUNT_KEY 1
+/// \brief The distinct value count (NDV) statistic.  The number of distinct
+///   values in the column.  Value type is int64 (when not approximate) or
+///   float64 (when approximate).
+#define ADBC_STATISTIC_DISTINCT_COUNT_NAME "adbc.statistic.distinct_count"
+/// \brief The dictionary-encoded name of the max byte width statistic.
+#define ADBC_STATISTIC_MAX_BYTE_WIDTH_KEY 2
+/// \brief The max byte width statistic.  The maximum size in bytes of a row
+///   in the column.  Value type is int64 (when not approximate) or float64
+///   (when approximate).
+///
+/// For example, this is the maximum length of a string for a string column.
+#define ADBC_STATISTIC_MAX_BYTE_WIDTH_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the max value statistic.
+#define ADBC_STATISTIC_MAX_VALUE_KEY 3
+/// \brief The max value statistic.  Value type is column-dependent.
+#define ADBC_STATISTIC_MAX_VALUE_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the min value statistic.
+#define ADBC_STATISTIC_MIN_VALUE_KEY 4
+/// \brief The min value statistic.  Value type is column-dependent.
+#define ADBC_STATISTIC_MIN_VALUE_NAME "adbc.statistic.byte_width"
+/// \brief The dictionary-encoded name of the null count statistic.
+#define ADBC_STATISTIC_NULL_COUNT_KEY 5
+/// \brief The null count statistic.  The number of values that are null in
+///   the column.  Value type is int64 (when not approximate) or float64
+///   (when approximate).
+#define ADBC_STATISTIC_NULL_COUNT_NAME "adbc.statistic.null_count"
+/// \brief The dictionary-encoded name of the row count statistic.
+#define ADBC_STATISTIC_ROW_COUNT_KEY 6
+/// \brief The row count statistic.  The number of rows in the column or
+///   table.  Value type is int64 (when not approximate) or float64 (when
+///   approximate).
+#define ADBC_STATISTIC_ROW_COUNT_NAME "adbc.statistic.row_count"
+/// @}
+
 /// \brief The name of the canonical option for whether autocommit is
 ///   enabled.
 ///
+/// The type is char*.
+///
 /// \see AdbcConnectionSetOption
 #define ADBC_CONNECTION_OPTION_AUTOCOMMIT "adbc.connection.autocommit"
 
 /// \brief The name of the canonical option for whether the current
 ///   connection should be restricted to being read-only.
 ///
+/// The type is char*.
+///
 /// \see AdbcConnectionSetOption
 #define ADBC_CONNECTION_OPTION_READ_ONLY "adbc.connection.readonly"
 
+/// \brief The name of the canonical option for the current catalog.
+///
+/// The type is char*.
+///
+/// \see AdbcConnectionGetOption
+/// \see AdbcConnectionSetOption
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_CONNECTION_OPTION_CURRENT_CATALOG "adbc.connection.catalog"
+
+/// \brief The name of the canonical option for the current schema.
+///
+/// The type is char*.
+///
+/// \see AdbcConnectionGetOption
+/// \see AdbcConnectionSetOption
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_CONNECTION_OPTION_CURRENT_DB_SCHEMA "adbc.connection.db_schema"
+
+/// \brief The name of the canonical option for making query execution
+///   nonblocking.
+///
+/// When enabled, AdbcStatementExecutePartitions will return
+/// partitions as soon as they are available, instead of returning
+/// them all at the end.  When there are no more to return, it will
+/// return an empty set of partitions.  AdbcStatementExecuteQuery and
+/// AdbcStatementExecuteSchema are not affected.
+///
+/// The default is ADBC_OPTION_VALUE_DISABLED.
+///
+/// The type is char*.
+///
+/// \see AdbcStatementSetOption
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_STATEMENT_OPTION_INCREMENTAL "adbc.statement.exec.incremental"
+
+/// \brief The name of the option for getting the progress of a query.
+///
+/// Progress is a value in [0.0, 1.0].
+///
+/// The type is double.
+///
+/// \see AdbcStatementGetOptionDouble
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_STATEMENT_OPTION_PROGRESS "adbc.statement.exec.progress"
+
 /// \brief The name of the canonical option for setting the isolation
 ///   level of a transaction.
 ///
@@ -357,6 +534,8 @@ struct ADBC_EXPORT AdbcError {
 /// isolation level is not supported by a driver, it should return an
 /// appropriate error.
 ///
+/// The type is char*.
+///
 /// \see AdbcConnectionSetOption
 #define ADBC_CONNECTION_OPTION_ISOLATION_LEVEL \
   "adbc.connection.transaction.isolation_level"
@@ -449,8 +628,12 @@ struct ADBC_EXPORT AdbcError {
 /// exist.  If the table exists but has a different schema,
 /// ADBC_STATUS_ALREADY_EXISTS should be raised.  Else, data should be
 /// appended to the target table.
+///
+/// The type is char*.
 #define ADBC_INGEST_OPTION_TARGET_TABLE "adbc.ingest.target_table"
 /// \brief Whether to create (the default) or append.
+///
+/// The type is char*.
 #define ADBC_INGEST_OPTION_MODE "adbc.ingest.mode"
 /// \brief Create the table and insert data; error if the table exists.
 #define ADBC_INGEST_OPTION_MODE_CREATE "adbc.ingest.mode.create"
@@ -458,6 +641,17 @@ struct ADBC_EXPORT AdbcError {
 ///   table does not exist (ADBC_STATUS_NOT_FOUND) or does not match
 ///   the schema of the data to append (ADBC_STATUS_ALREADY_EXISTS).
 #define ADBC_INGEST_OPTION_MODE_APPEND "adbc.ingest.mode.append"
+/// \brief Create the table and insert data; drop the original table
+///   if it already exists.
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_INGEST_OPTION_MODE_REPLACE "adbc.ingest.mode.replace"
+/// \brief Insert data; create the table if it does not exist, or
+///   error if the table exists, but the schema does not match the
+///   schema of the data to append (ADBC_STATUS_ALREADY_EXISTS).
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_INGEST_OPTION_MODE_CREATE_APPEND "adbc.ingest.mode.create_append"
 
 /// @}
 
@@ -667,8 +861,105 @@ struct ADBC_EXPORT AdbcDriver {
                                          struct AdbcError*);
   AdbcStatusCode (*StatementSetSubstraitPlan)(struct AdbcStatement*, const uint8_t*,
                                               size_t, struct AdbcError*);
+
+  /// \defgroup adbc-1.1.0 ADBC API Revision 1.1.0
+  ///
+  /// Functions added in ADBC 1.1.0.  For backwards compatibility,
+  /// these members must not be accessed unless the version passed to
+  /// the AdbcDriverInitFunc is greater than or equal to
+  /// ADBC_VERSION_1_1_0.
+  ///
+  /// For a 1.0.0 driver being loaded by a 1.1.0 driver manager: the
+  /// 1.1.0 manager will allocate the new, expanded AdbcDriver struct
+  /// and attempt to have the driver initialize it with
+  /// ADBC_VERSION_1_1_0.  This must return an error, after which the
+  /// driver will try again with ADBC_VERSION_1_0_0.  The driver must
+  /// not access the new fields, which will carry undefined values.
+  ///
+  /// For a 1.1.0 driver being loaded by a 1.0.0 driver manager: the
+  /// 1.0.0 manager will allocate the old AdbcDriver struct and
+  /// attempt to have the driver initialize it with
+  /// ADBC_VERSION_1_0_0.  The driver must not access the new fields,
+  /// and should initialize the old fields.
+  ///
+  /// @{
+
+  AdbcStatusCode (*DatabaseGetOption)(struct AdbcDatabase*, const char*, char*, size_t*,
+                                      struct AdbcError*);
+  AdbcStatusCode (*DatabaseGetOptionBytes)(struct AdbcDatabase*, const char*, uint8_t*,
+                                           size_t*, struct AdbcError*);
+  AdbcStatusCode (*DatabaseGetOptionInt)(struct AdbcDatabase*, const char*, int64_t*,
+                                         struct AdbcError*);
+  AdbcStatusCode (*DatabaseGetOptionDouble)(struct AdbcDatabase*, const char*, double*,
+                                            struct AdbcError*);
+  AdbcStatusCode (*DatabaseSetOptionBytes)(struct AdbcDatabase*, const char*,
+                                           const uint8_t*, size_t, struct AdbcError*);
+  AdbcStatusCode (*DatabaseSetOptionInt)(struct AdbcDatabase*, const char*, int64_t,
+                                         struct AdbcError*);
+  AdbcStatusCode (*DatabaseSetOptionDouble)(struct AdbcDatabase*, const char*, double,
+                                            struct AdbcError*);
+
+  AdbcStatusCode (*ConnectionCancel)(struct AdbcConnection*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetOption)(struct AdbcConnection*, const char*, char*,
+                                        size_t*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetOptionBytes)(struct AdbcDatabase*, const char*, uint8_t*,
+                                             size_t*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetOptionInt)(struct AdbcConnection*, const char*, int64_t*,
+                                           struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetOptionDouble)(struct AdbcConnection*, const char*,
+                                              double*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetStatistics)(struct AdbcConnection*, const char*,
+                                            const char*, const char*, char,
+                                            struct ArrowArrayStream*, struct AdbcError*);
+  AdbcStatusCode (*ConnectionGetStatisticNames)(struct AdbcConnection*,
+                                                struct ArrowArrayStream*,
+                                                struct AdbcError*);
+  AdbcStatusCode (*ConnectionSetOptionBytes)(struct AdbcDatabase*, const char*,
+                                             const uint8_t*, size_t, struct AdbcError*);
+  AdbcStatusCode (*ConnectionSetOptionInt)(struct AdbcConnection*, const char*, int64_t,
+                                           struct AdbcError*);
+  AdbcStatusCode (*ConnectionSetOptionDouble)(struct AdbcConnection*, const char*, double,
+                                              struct AdbcError*);
+
+  AdbcStatusCode (*StatementCancel)(struct AdbcStatement*, struct AdbcError*);
+  AdbcStatusCode (*StatementExecuteSchema)(struct AdbcStatement*, struct ArrowSchema*,
+                                           struct AdbcError*);
+  AdbcStatusCode (*StatementGetOption)(struct AdbcStatement*, const char*, char*, size_t*,
+                                       struct AdbcError*);
+  AdbcStatusCode (*StatementGetOptionBytes)(struct AdbcDatabase*, const char*, uint8_t*,
+                                            size_t*, struct AdbcError*);
+  AdbcStatusCode (*StatementGetOptionInt)(struct AdbcStatement*, const char*, int64_t*,
+                                          struct AdbcError*);
+  AdbcStatusCode (*StatementGetOptionDouble)(struct AdbcStatement*, const char*, double*,
+                                             struct AdbcError*);
+  AdbcStatusCode (*StatementSetOptionBytes)(struct AdbcDatabase*, const char*,
+                                            const uint8_t*, size_t, struct AdbcError*);
+  AdbcStatusCode (*StatementSetOptionInt)(struct AdbcStatement*, const char*, int64_t,
+                                          struct AdbcError*);
+  AdbcStatusCode (*StatementSetOptionDouble)(struct AdbcStatement*, const char*, double,
+                                             struct AdbcError*);
+
+  /// @}
 };
 
+/// \brief The size of the AdbcDriver structure in ADBC 1.0.0.
+/// Drivers written for ADBC 1.1.0 and later should never touch more
+/// than this portion of an AdbcDriver struct when given
+/// ADBC_VERSION_1_0_0.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_DRIVER_1_0_0_SIZE (offsetof(struct AdbcDriver, DatabaseGetOption))
+
+/// \brief The size of the AdbcDriver structure in ADBC 1.1.0.
+/// Drivers written for ADBC 1.1.0 and later should never touch more
+/// than this portion of an AdbcDriver struct when given
+/// ADBC_VERSION_1_1_0.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+#define ADBC_DRIVER_1_1_0_SIZE (sizeof(struct AdbcDriver))
+
 /// @}
 
 /// \addtogroup adbc-database
@@ -684,16 +975,192 @@ struct ADBC_EXPORT AdbcDriver {
 ADBC_EXPORT
 AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError* error);
 
+/// \brief Get a string option of the database.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call GetOption
+/// concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value (including the null terminator) to buffer and set
+/// length to the size of the actual value.  If the buffer is too
+/// small, no data will be written and length will be set to the
+/// required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The length of value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase* database, const char* key,
+                                     char* value, size_t* length,
+                                     struct AdbcError* error);
+
+/// \brief Get a bytestring option of the database.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call
+/// GetOptionBytes concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value to buffer and set length to the size of the
+/// actual value.  If the buffer is too small, no data will be written
+/// and length will be set to the required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcDatabaseGetOptionBytes(struct AdbcDatabase* database, const char* key,
+                                          uint8_t* value, size_t* length,
+                                          struct AdbcError* error);
+
+/// \brief Get an integer option of the database.
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the integer
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcDatabaseGetOptionInt(struct AdbcDatabase* database, const char* key,
+                                        int64_t* value, struct AdbcError* error);
+
+/// \brief Get a double option of the database.
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the double
+/// representation of an integer option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcDatabaseGetOptionDouble(struct AdbcDatabase* database, const char* key,
+                                           double* value, struct AdbcError* error);
+
 /// \brief Set a char* option.
 ///
 /// Options may be set before AdbcDatabaseInit.  Some drivers may
 /// support setting options after initialization as well.
 ///
+/// \param[in] database The database.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
 /// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
 ADBC_EXPORT
 AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase* database, const char* key,
                                      const char* value, struct AdbcError* error);
 
+/// \brief Set a bytestring option on a database.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[in] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcDatabaseSetOptionBytes(struct AdbcDatabase* database, const char* key,
+                                          const uint8_t* value, size_t length,
+                                          struct AdbcError* error);
+
+/// \brief Set an integer option on a database.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcDatabaseSetOptionInt(struct AdbcDatabase* database, const char* key,
+                                        int64_t value, struct AdbcError* error);
+
+/// \brief Set a double option on a database.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] database The database.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcDatabaseSetOptionDouble(struct AdbcDatabase* database, const char* key,
+                                           double value, struct AdbcError* error);
+
 /// \brief Finish setting options and initialize the database.
 ///
 /// Some drivers may support setting options after initialization
@@ -730,11 +1197,68 @@ AdbcStatusCode AdbcConnectionNew(struct AdbcConnection* connection,
 /// Options may be set before AdbcConnectionInit.  Some drivers may
 /// support setting options after initialization as well.
 ///
+/// \param[in] connection The database connection.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
 /// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
 ADBC_EXPORT
 AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection* connection, const char* key,
                                        const char* value, struct AdbcError* error);
 
+/// \brief Set a bytestring option on a connection.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The connection.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[in] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionSetOptionBytes(struct AdbcConnection* connection,
+                                            const char* key, const uint8_t* value,
+                                            size_t length, struct AdbcError* error);
+
+/// \brief Set an integer option.
+///
+/// Options may be set before AdbcConnectionInit.  Some drivers may
+/// support setting options after initialization as well.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionSetOptionInt(struct AdbcConnection* connection,
+                                          const char* key, int64_t value,
+                                          struct AdbcError* error);
+
+/// \brief Set a double option.
+///
+/// Options may be set before AdbcConnectionInit.  Some drivers may
+/// support setting options after initialization as well.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionSetOptionDouble(struct AdbcConnection* connection,
+                                             const char* key, double value,
+                                             struct AdbcError* error);
+
 /// \brief Finish setting options and initialize the connection.
 ///
 /// Some drivers may support setting options after initialization
@@ -752,6 +1276,29 @@ ADBC_EXPORT
 AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection,
                                      struct AdbcError* error);
 
+/// \brief Cancel the in-progress operation on a connection.
+///
+/// This can be called during AdbcConnectionGetObjects (or similar),
+/// or while consuming an ArrowArrayStream returned from such.
+/// Calling this function should make the other functions return
+/// ADBC_STATUS_CANCELLED (from ADBC functions) or ECANCELED (from
+/// methods of ArrowArrayStream).
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+///
+/// \param[in] connection The connection to cancel.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+///
+/// \return ADBC_STATUS_INVALID_STATE if there is no operation to cancel.
+/// \return ADBC_STATUS_UNKNOWN if the operation could not be cancelled.
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection* connection,
+                                    struct AdbcError* error);
+
 /// \defgroup adbc-connection-metadata Metadata
 /// Functions for retrieving metadata about the database.
 ///
@@ -765,6 +1312,8 @@ AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection,
 /// concurrent active statements and it must execute a SQL query
 /// internally in order to implement the metadata function).
 ///
+/// This AdbcConnection must outlive the returned ArrowArrayStream.
+///
 /// Some functions accept "search pattern" arguments, which are
 /// strings that can contain the special character "%" to match zero
 /// or more characters, or "_" to match exactly one character.  (See
@@ -799,6 +1348,10 @@ AdbcStatusCode AdbcConnectionRelease(struct AdbcConnection* connection,
 /// for ADBC usage.  Drivers/vendors will ignore requests for
 /// unrecognized codes (the row will be omitted from the result).
 ///
+/// Since ADBC 1.1.0: the range [500, 1_000) is reserved for "XDBC"
+/// information, which is the same metadata provided by the same info
+/// code range in the Arrow Flight SQL GetSqlInfo RPC.
+///
 /// \param[in] connection The connection to query.
 /// \param[in] info_codes A list of metadata codes to fetch, or NULL
 ///   to fetch all.
@@ -891,6 +1444,8 @@ AdbcStatusCode AdbcConnectionGetInfo(struct AdbcConnection* connection,
 /// | fk_table                 | utf8 not null           |
 /// | fk_column_name           | utf8 not null           |
 ///
+/// This AdbcConnection must outlive the returned ArrowArrayStream.
+///
 /// \param[in] connection The database connection.
 /// \param[in] depth The level of nesting to display. If 0, display
 ///   all levels. If 1, display only catalogs (i.e.  catalog_schemas
@@ -922,6 +1477,215 @@ AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int d
                                         struct ArrowArrayStream* out,
                                         struct AdbcError* error);
 
+/// \brief Get a string option of the connection.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call GetOption
+/// concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value (including the null terminator) to buffer and set
+/// length to the size of the actual value.  If the buffer is too
+/// small, no data will be written and length will be set to the
+/// required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The length of value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const char* key,
+                                       char* value, size_t* length,
+                                       struct AdbcError* error);
+
+/// \brief Get a bytestring option of the connection.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call
+/// GetOptionBytes concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value to buffer and set length to the size of the
+/// actual value.  If the buffer is too small, no data will be written
+/// and length will be set to the required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The connection.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcConnectionGetOptionBytes(struct AdbcConnection* connection,
+                                            const char* key, uint8_t* value,
+                                            size_t* length, struct AdbcError* error);
+
+/// \brief Get an integer option of the connection.
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcConnectionGetOptionInt(struct AdbcConnection* connection,
+                                          const char* key, int64_t* value,
+                                          struct AdbcError* error);
+
+/// \brief Get a double option of the connection.
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcConnectionGetOptionDouble(struct AdbcConnection* connection,
+                                             const char* key, double* value,
+                                             struct AdbcError* error);
+
+/// \brief Get statistics about the data distribution of table(s).
+///
+/// The result is an Arrow dataset with the following schema:
+///
+/// | Field Name               | Field Type                       |
+/// |--------------------------|----------------------------------|
+/// | catalog_name             | utf8                             |
+/// | catalog_db_schemas       | list<DB_SCHEMA_SCHEMA>           |
+///
+/// DB_SCHEMA_SCHEMA is a Struct with fields:
+///
+/// | Field Name               | Field Type                       |
+/// |--------------------------|----------------------------------|
+/// | db_schema_name           | utf8                             |
+/// | db_schema_functions      | list<STATISTICS_SCHEMA>          |
+///
+/// STATISTICS_SCHEMA is a Struct with fields:
+///
+/// | Field Name               | Field Type                       | Comments |
+/// |--------------------------|----------------------------------| -------- |
+/// | table_name               | utf8 not null                    |          |
+/// | column_name              | utf8                             | (1)      |
+/// | statistic_key            | int16 not null                   | (2)      |
+/// | statistic_value          | VALUE_SCHEMA not null            |          |
+/// | statistic_is_approximate | bool not null                    | (3)      |
+///
+/// 1. If null, then the statistic applies to the entire table.
+/// 2. A dictionary-encoded statistic name (although we do not use the Arrow
+///    dictionary type). Values in [0, 1024) are reserved for ADBC.  Other
+///    values are for implementation-specific statistics.  For the definitions
+///    of predefined statistic types, see \ref adbc-table-statistics.  To get
+///    driver-specific statistic names, use AdbcConnectionGetStatisticNames.
+/// 3. If true, then the value is approximate or best-effort.
+///
+/// VALUE_SCHEMA is a dense union with members:
+///
+/// | Field Name               | Field Type                       |
+/// |--------------------------|----------------------------------|
+/// | int64                    | int64                            |
+/// | uint64                   | uint64                           |
+/// | float64                  | float64                          |
+/// | decimal256               | decimal256                       |
+/// | binary                   | binary                           |
+///
+/// This AdbcConnection must outlive the returned ArrowArrayStream.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[in] catalog The catalog (or nullptr).  May be a search
+///   pattern (see section documentation).
+/// \param[in] db_schema The database schema (or nullptr).  May be a
+///   search pattern (see section documentation).
+/// \param[in] table_name The table name (or nullptr).  May be a
+///   search pattern (see section documentation).
+/// \param[in] approximate If zero, request exact values of
+///   statistics, else allow for best-effort, approximate, or cached
+///   values.  The database may return approximate values regardless,
+///   as indicated in the result.  Requesting exact values may be
+///   expensive or unsupported.
+/// \param[out] out The result set.
+/// \param[out] error Error details, if an error occurs.
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionGetStatistics(struct AdbcConnection* connection,
+                                           const char* catalog, const char* db_schema,
+                                           const char* table_name, char approximate,
+                                           struct ArrowArrayStream* out,
+                                           struct AdbcError* error);
+
+/// \brief Get the names of statistics specific to this driver.
+///
+/// The result is an Arrow dataset with the following schema:
+///
+/// Field Name     | Field Type
+/// ---------------|----------------
+/// statistic_name | utf8 not null
+/// statistic_key  | int16 not null
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] connection The database connection.
+/// \param[out] out The result set.
+/// \param[out] error Error details, if an error occurs.
+ADBC_EXPORT
+AdbcStatusCode AdbcConnectionGetStatisticNames(struct AdbcConnection* connection,
+                                               struct ArrowArrayStream* out,
+                                               struct AdbcError* error);
+
 /// \brief Get the Arrow schema of a table.
 ///
 /// \param[in] connection The database connection.
@@ -945,6 +1709,8 @@ AdbcStatusCode AdbcConnectionGetTableSchema(struct AdbcConnection* connection,
 /// ---------------|--------------
 /// table_type     | utf8 not null
 ///
+/// This AdbcConnection must outlive the returned ArrowArrayStream.
+///
 /// \param[in] connection The database connection.
 /// \param[out] out The result set.
 /// \param[out] error Error details, if an error occurs.
@@ -973,6 +1739,8 @@ AdbcStatusCode AdbcConnectionGetTableTypes(struct AdbcConnection* connection,
 ///
 /// A partition can be retrieved from AdbcPartitions.
 ///
+/// This AdbcConnection must outlive the returned ArrowArrayStream.
+///
 /// \param[in] connection The connection to use.  This does not have
 ///   to be the same connection that the partition was created on.
 /// \param[in] serialized_partition The partition descriptor.
@@ -1042,7 +1810,11 @@ AdbcStatusCode AdbcStatementRelease(struct AdbcStatement* statement,
 
 /// \brief Execute a statement and get the results.
 ///
-/// This invalidates any prior result sets.
+/// This invalidates any prior result sets.  This AdbcStatement must
+/// outlive the returned ArrowArrayStream.
+///
+/// Since ADBC 1.1.0: releasing the returned ArrowArrayStream without
+/// consuming it fully is equivalent to calling AdbcStatementCancel.
 ///
 /// \param[in] statement The statement to execute.
 /// \param[out] out The results. Pass NULL if the client does not
@@ -1056,6 +1828,28 @@ AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement* statement,
                                          struct ArrowArrayStream* out,
                                          int64_t* rows_affected, struct AdbcError* error);
 
+/// \brief Get the schema of the result set of a query without
+///   executing it.
+///
+/// This invalidates any prior result sets.
+///
+/// Depending on the driver, this may require first executing
+/// AdbcStatementPrepare.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+///
+/// \param[in] statement The statement to execute.
+/// \param[out] out The result schema.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+///
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the driver does not support this.
+ADBC_EXPORT
+AdbcStatusCode AdbcStatementExecuteSchema(struct AdbcStatement* statement,
+                                          struct ArrowSchema* schema,
+                                          struct AdbcError* error);
+
 /// \brief Turn this statement into a prepared statement to be
 ///   executed multiple times.
 ///
@@ -1138,6 +1932,157 @@ AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement* statement,
                                        struct ArrowArrayStream* stream,
                                        struct AdbcError* error);
 
+/// \brief Cancel execution of an in-progress query.
+///
+/// This can be called during AdbcStatementExecuteQuery (or similar),
+/// or while consuming an ArrowArrayStream returned from such.
+/// Calling this function should make the other functions return
+/// ADBC_STATUS_CANCELLED (from ADBC functions) or ECANCELED (from
+/// methods of ArrowArrayStream).
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+///
+/// \param[in] statement The statement to cancel.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+///
+/// \return ADBC_STATUS_INVALID_STATE if there is no query to cancel.
+/// \return ADBC_STATUS_UNKNOWN if the query could not be cancelled.
+ADBC_EXPORT
+AdbcStatusCode AdbcStatementCancel(struct AdbcStatement* statement,
+                                   struct AdbcError* error);
+
+/// \brief Get a string option of the statement.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call GetOption
+/// concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value (including the null terminator) to buffer and set
+/// length to the size of the actual value.  If the buffer is too
+/// small, no data will be written and length will be set to the
+/// required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The length of value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement* statement, const char* key,
+                                      char* value, size_t* length,
+                                      struct AdbcError* error);
+
+/// \brief Get a bytestring option of the statement.
+///
+/// This must always be thread-safe (other operations are not), though
+/// given the semantics here, it is not recommended to call
+/// GetOptionBytes concurrently with itself.
+///
+/// length must be provided and must be the size of the buffer pointed
+/// to by value.  If there is sufficient space, the driver will copy
+/// the option value to buffer and set length to the size of the
+/// actual value.  If the buffer is too small, no data will be written
+/// and length will be set to the required length.
+///
+/// In other words:
+///
+/// - If output length <= input length, value will contain a value
+///   with length bytes.
+/// - If output length > input length, nothing has been written to
+///   value.
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[in,out] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcStatementGetOptionBytes(struct AdbcStatement* statement,
+                                           const char* key, uint8_t* value,
+                                           size_t* length, struct AdbcError* error);
+
+/// \brief Get an integer option of the statement.
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcStatementGetOptionInt(struct AdbcStatement* statement, const char* key,
+                                         int64_t* value, struct AdbcError* error);
+
+/// \brief Get a double option of the statement.
+///
+/// This must always be thread-safe (other operations are not).
+///
+/// For standard options, drivers must always support getting the
+/// option value (if they support getting option values at all) via
+/// the type specified in the option.  (For example, an option set via
+/// SetOptionDouble must be retrievable via GetOptionDouble.)  Drivers
+/// may also support getting a converted option value via other
+/// getters if needed.  (For example, getting the string
+/// representation of a double option.)
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to get.
+/// \param[out] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_FOUND if the option is not recognized.
+AdbcStatusCode AdbcStatementGetOptionDouble(struct AdbcStatement* statement,
+                                            const char* key, double* value,
+                                            struct AdbcError* error);
+
 /// \brief Get the schema for bound parameters.
 ///
 /// This retrieves an Arrow schema describing the number, names, and
@@ -1159,10 +2104,61 @@ AdbcStatusCode AdbcStatementGetParameterSchema(struct AdbcStatement* statement,
                                                struct AdbcError* error);
 
 /// \brief Set a string option on a statement.
+/// \param[in] statement The statement.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized.
 ADBC_EXPORT
 AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement* statement, const char* key,
                                       const char* value, struct AdbcError* error);
 
+/// \brief Set a bytestring option on a statement.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[in] length The option value length.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcStatementSetOptionBytes(struct AdbcStatement* statement,
+                                           const char* key, const uint8_t* value,
+                                           size_t length, struct AdbcError* error);
+
+/// \brief Set an integer option on a statement.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcStatementSetOptionInt(struct AdbcStatement* statement, const char* key,
+                                         int64_t value, struct AdbcError* error);
+
+/// \brief Set a double option on a statement.
+///
+/// \since ADBC API revision 1.1.0
+/// \addtogroup adbc-1.1.0
+/// \param[in] statement The statement.
+/// \param[in] key The option to set.
+/// \param[in] value The option value.
+/// \param[out] error An optional location to return an error
+///   message if necessary.
+/// \return ADBC_STATUS_NOT_IMPLEMENTED if the option is not recognized
+ADBC_EXPORT
+AdbcStatusCode AdbcStatementSetOptionDouble(struct AdbcStatement* statement,
+                                            const char* key, double value,
+                                            struct AdbcError* error);
+
 /// \addtogroup adbc-statement-partition
 /// @{
 
@@ -1198,7 +2194,15 @@ AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement* statement,
 ///   driver.
 ///
 /// Although drivers may choose any name for this function, the
-/// recommended name is "AdbcDriverInit".
+/// recommended name is "AdbcDriverInit", or a name derived from the
+/// name of the driver's shared library as follows: remove the 'lib'
+/// prefix (on Unix systems) and all file extensions, then PascalCase
+/// the driver name, append Init, and prepend Adbc (if not already
+/// there).  For example:
+///
+/// - libadbc_driver_sqlite.so.2.0.0 -> AdbcDriverSqliteInit
+/// - adbc_driver_sqlite.dll -> AdbcDriverSqliteInit
+/// - proprietary_driver.dll -> AdbcProprietaryDriverInit
 ///
 /// \param[in] version The ADBC revision to attempt to initialize (see
 ///   ADBC_VERSION_1_0_0).
diff --git a/go/adbc/drivermgr/adbc_driver_manager.cc b/go/adbc/drivermgr/adbc_driver_manager.cc
index c63560a4..8486bf1a 100644
--- a/go/adbc/drivermgr/adbc_driver_manager.cc
+++ b/go/adbc/drivermgr/adbc_driver_manager.cc
@@ -19,6 +19,7 @@
 #include <adbc.h>
 
 #include <algorithm>
+#include <array>
 #include <cstring>
 #include <string>
 #include <unordered_map>
@@ -129,11 +130,41 @@ static AdbcStatusCode ReleaseDriver(struct AdbcDriver* driver, struct AdbcError*
 
 // Default stubs
 
+AdbcStatusCode DatabaseGetOption(struct AdbcDatabase* database, const char* key,
+                                 char* value, size_t* length, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode DatabaseGetOptionInt(struct AdbcDatabase* database, const char* key,
+                                    int64_t* value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode DatabaseGetOptionDouble(struct AdbcDatabase* database, const char* key,
+                                       double* value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode DatabaseSetOption(struct AdbcDatabase* database, const char* key,
                                  const char* value, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode DatabaseSetOptionInt(struct AdbcDatabase* database, const char* key,
+                                    int64_t value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode DatabaseSetOptionDouble(struct AdbcDatabase* database, const char* key,
+                                       double value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode ConnectionCancel(struct AdbcConnection* connection,
+                                struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode ConnectionCommit(struct AdbcConnection*, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
@@ -150,6 +181,22 @@ AdbcStatusCode ConnectionGetObjects(struct AdbcConnection*, int, const char*, co
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode ConnectionGetOption(struct AdbcConnection* connection, const char* key,
+                                   char* value, size_t* length, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode ConnectionGetOptionInt(struct AdbcConnection* connection, const char* key,
+                                      int64_t* value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode ConnectionGetOptionDouble(struct AdbcConnection* connection,
+                                         const char* key, double* value,
+                                         struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode ConnectionGetTableSchema(struct AdbcConnection*, const char*, const char*,
                                         const char*, struct ArrowSchema*,
                                         struct AdbcError* error) {
@@ -178,11 +225,26 @@ AdbcStatusCode ConnectionSetOption(struct AdbcConnection*, const char*, const ch
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode ConnectionSetOptionInt(struct AdbcConnection* connection, const char* key,
+                                      int64_t value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode ConnectionSetOptionDouble(struct AdbcConnection* connection,
+                                         const char* key, double value,
+                                         struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode StatementBind(struct AdbcStatement*, struct ArrowArray*,
                              struct ArrowSchema*, struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode StatementCancel(struct AdbcStatement* statement, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode StatementExecutePartitions(struct AdbcStatement* statement,
                                           struct ArrowSchema* schema,
                                           struct AdbcPartitions* partitions,
@@ -191,6 +253,27 @@ AdbcStatusCode StatementExecutePartitions(struct AdbcStatement* statement,
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode StatementExecuteSchema(struct AdbcStatement* statement,
+                                      struct ArrowSchema* schema,
+                                      struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode StatementGetOption(struct AdbcStatement* statement, const char* key,
+                                  char* value, size_t* length, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode StatementGetOptionInt(struct AdbcStatement* statement, const char* key,
+                                     int64_t* value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode StatementGetOptionDouble(struct AdbcStatement* statement, const char* key,
+                                        double* value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode StatementGetParameterSchema(struct AdbcStatement* statement,
                                            struct ArrowSchema* schema,
                                            struct AdbcError* error) {
@@ -206,6 +289,16 @@ AdbcStatusCode StatementSetOption(struct AdbcStatement*, const char*, const char
   return ADBC_STATUS_NOT_IMPLEMENTED;
 }
 
+AdbcStatusCode StatementSetOptionInt(struct AdbcStatement* statement, const char* key,
+                                     int64_t value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
+AdbcStatusCode StatementSetOptionDouble(struct AdbcStatement* statement, const char* key,
+                                        double value, struct AdbcError* error) {
+  return ADBC_STATUS_NOT_IMPLEMENTED;
+}
+
 AdbcStatusCode StatementSetSqlQuery(struct AdbcStatement*, const char*,
                                     struct AdbcError* error) {
   return ADBC_STATUS_NOT_IMPLEMENTED;
@@ -219,6 +312,8 @@ AdbcStatusCode StatementSetSubstraitPlan(struct AdbcStatement*, const uint8_t*,
 /// Temporary state while the database is being configured.
 struct TempDatabase {
   std::unordered_map<std::string, std::string> options;
+  std::unordered_map<std::string, int64_t> int_options;
+  std::unordered_map<std::string, double> double_options;
   std::string driver;
   // Default name (see adbc.h)
   std::string entrypoint = "AdbcDriverInit";
@@ -228,6 +323,8 @@ struct TempDatabase {
 /// Temporary state while the database is being configured.
 struct TempConnection {
   std::unordered_map<std::string, std::string> options;
+  std::unordered_map<std::string, int64_t> int_options;
+  std::unordered_map<std::string, double> double_options;
 };
 }  // namespace
 
@@ -240,6 +337,63 @@ AdbcStatusCode AdbcDatabaseNew(struct AdbcDatabase* database, struct AdbcError*
   return ADBC_STATUS_OK;
 }
 
+AdbcStatusCode AdbcDatabaseGetOption(struct AdbcDatabase* database, const char* key,
+                                     char* value, size_t* length,
+                                     struct AdbcError* error) {
+  if (database->private_driver) {
+    return database->private_driver->DatabaseGetOption(database, key, value, length,
+                                                       error);
+  }
+  const auto* args = reinterpret_cast<const TempDatabase*>(database->private_data);
+  const std::string* result = nullptr;
+  if (std::strcmp(key, "driver") == 0) {
+    result = &args->driver;
+  } else if (std::strcmp(key, "entrypoint") == 0) {
+    result = &args->entrypoint;
+  } else {
+    const auto it = args->options.find(key);
+    if (it == args->options.end()) {
+      return ADBC_STATUS_NOT_FOUND;
+    }
+    result = &it->second;
+  }
+
+  if (*length <= result->size() + 1) {
+    // Enough space
+    std::memcpy(value, result->c_str(), result->size() + 1);
+  }
+  *length = result->size() + 1;
+  return ADBC_STATUS_OK;
+}
+
+AdbcStatusCode AdbcDatabaseGetOptionInt(struct AdbcDatabase* database, const char* key,
+                                        int64_t* value, struct AdbcError* error) {
+  if (database->private_driver) {
+    return database->private_driver->DatabaseGetOptionInt(database, key, value, error);
+  }
+  const auto* args = reinterpret_cast<const TempDatabase*>(database->private_data);
+  const auto it = args->int_options.find(key);
+  if (it == args->int_options.end()) {
+    return ADBC_STATUS_NOT_FOUND;
+  }
+  *value = it->second;
+  return ADBC_STATUS_OK;
+}
+
+AdbcStatusCode AdbcDatabaseGetOptionDouble(struct AdbcDatabase* database, const char* key,
+                                           double* value, struct AdbcError* error) {
+  if (database->private_driver) {
+    return database->private_driver->DatabaseGetOptionDouble(database, key, value, error);
+  }
+  const auto* args = reinterpret_cast<const TempDatabase*>(database->private_data);
+  const auto it = args->double_options.find(key);
+  if (it == args->double_options.end()) {
+    return ADBC_STATUS_NOT_FOUND;
+  }
+  *value = it->second;
+  return ADBC_STATUS_OK;
+}
+
 AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase* database, const char* key,
                                      const char* value, struct AdbcError* error) {
   if (database->private_driver) {
@@ -257,6 +411,28 @@ AdbcStatusCode AdbcDatabaseSetOption(struct AdbcDatabase* database, const char*
   return ADBC_STATUS_OK;
 }
 
+AdbcStatusCode AdbcDatabaseSetOptionInt(struct AdbcDatabase* database, const char* key,
+                                        int64_t value, struct AdbcError* error) {
+  if (database->private_driver) {
+    return database->private_driver->DatabaseSetOptionInt(database, key, value, error);
+  }
+
+  TempDatabase* args = reinterpret_cast<TempDatabase*>(database->private_data);
+  args->int_options[key] = value;
+  return ADBC_STATUS_OK;
+}
+
+AdbcStatusCode AdbcDatabaseSetOptionDouble(struct AdbcDatabase* database, const char* key,
+                                           double value, struct AdbcError* error) {
+  if (database->private_driver) {
+    return database->private_driver->DatabaseSetOptionDouble(database, key, value, error);
+  }
+
+  TempDatabase* args = reinterpret_cast<TempDatabase*>(database->private_data);
+  args->double_options[key] = value;
+  return ADBC_STATUS_OK;
+}
+
 AdbcStatusCode AdbcDriverManagerDatabaseSetInitFunc(struct AdbcDatabase* database,
                                                     AdbcDriverInitFunc init_func,
                                                     struct AdbcError* error) {
@@ -313,25 +489,40 @@ AdbcStatusCode AdbcDatabaseInit(struct AdbcDatabase* database, struct AdbcError*
     database->private_driver = nullptr;
     return status;
   }
-  for (const auto& option : args->options) {
+  auto options = std::move(args->options);
+  auto int_options = std::move(args->int_options);
+  auto double_options = std::move(args->double_options);
+  delete args;
+
+  for (const auto& option : options) {
     status = database->private_driver->DatabaseSetOption(database, option.first.c_str(),
                                                          option.second.c_str(), error);
-    if (status != ADBC_STATUS_OK) {
-      delete args;
-      // Release the database
-      std::ignore = database->private_driver->DatabaseRelease(database, error);
-      if (database->private_driver->release) {
-        database->private_driver->release(database->private_driver, error);
-      }
-      delete database->private_driver;
-      database->private_driver = nullptr;
-      // Should be redundant, but ensure that AdbcDatabaseRelease
-      // below doesn't think that it contains a TempDatabase
-      database->private_data = nullptr;
-      return status;
+    if (status != ADBC_STATUS_OK) break;
+  }
+  for (const auto& option : int_options) {
+    status = database->private_driver->DatabaseSetOptionInt(
+        database, option.first.c_str(), option.second, error);
+    if (status != ADBC_STATUS_OK) break;
+  }
+  for (const auto& option : double_options) {
+    status = database->private_driver->DatabaseSetOptionDouble(
+        database, option.first.c_str(), option.second, error);
+    if (status != ADBC_STATUS_OK) break;
+  }
+
+  if (status != ADBC_STATUS_OK) {
+    // Release the database
+    std::ignore = database->private_driver->DatabaseRelease(database, error);
+    if (database->private_driver->release) {
+      database->private_driver->release(database->private_driver, error);
     }
+    delete database->private_driver;
+    database->private_driver = nullptr;
+    // Should be redundant, but ensure that AdbcDatabaseRelease
+    // below doesn't think that it contains a TempDatabase
+    database->private_data = nullptr;
+    return status;
   }
-  delete args;
   return database->private_driver->DatabaseInit(database, error);
 }
 
@@ -356,6 +547,14 @@ AdbcStatusCode AdbcDatabaseRelease(struct AdbcDatabase* database,
   return status;
 }
 
+AdbcStatusCode AdbcConnectionCancel(struct AdbcConnection* connection,
+                                    struct AdbcError* error) {
+  if (!connection->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return connection->private_driver->ConnectionCancel(connection, error);
+}
+
 AdbcStatusCode AdbcConnectionCommit(struct AdbcConnection* connection,
                                     struct AdbcError* error) {
   if (!connection->private_driver) {
@@ -389,6 +588,72 @@ AdbcStatusCode AdbcConnectionGetObjects(struct AdbcConnection* connection, int d
       error);
 }
 
+AdbcStatusCode AdbcConnectionGetOption(struct AdbcConnection* connection, const char* key,
+                                       char* value, size_t* length,
+                                       struct AdbcError* error) {
+  if (!connection->private_data) {
+    SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first");
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  if (!connection->private_driver) {
+    // Init not yet called, get the saved option
+    const auto* args = reinterpret_cast<const TempConnection*>(connection->private_data);
+    const auto it = args->options.find(key);
+    if (it == args->options.end()) {
+      return ADBC_STATUS_NOT_FOUND;
+    }
+    if (*length >= it->second.size() + 1) {
+      std::memcpy(value, it->second.c_str(), it->second.size() + 1);
+    }
+    *length = it->second.size() + 1;
+    return ADBC_STATUS_OK;
+  }
+  return connection->private_driver->ConnectionGetOption(connection, key, value, length,
+                                                         error);
+}
+
+AdbcStatusCode AdbcConnectionGetOptionInt(struct AdbcConnection* connection,
+                                          const char* key, int64_t* value,
+                                          struct AdbcError* error) {
+  if (!connection->private_data) {
+    SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first");
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  if (!connection->private_driver) {
+    // Init not yet called, get the saved option
+    const auto* args = reinterpret_cast<const TempConnection*>(connection->private_data);
+    const auto it = args->int_options.find(key);
+    if (it == args->int_options.end()) {
+      return ADBC_STATUS_NOT_FOUND;
+    }
+    *value = it->second;
+    return ADBC_STATUS_OK;
+  }
+  return connection->private_driver->ConnectionGetOptionInt(connection, key, value,
+                                                            error);
+}
+
+AdbcStatusCode AdbcConnectionGetOptionDouble(struct AdbcConnection* connection,
+                                             const char* key, double* value,
+                                             struct AdbcError* error) {
+  if (!connection->private_data) {
+    SetError(error, "AdbcConnectionGetOption: must AdbcConnectionNew first");
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  if (!connection->private_driver) {
+    // Init not yet called, get the saved option
+    const auto* args = reinterpret_cast<const TempConnection*>(connection->private_data);
+    const auto it = args->double_options.find(key);
+    if (it == args->double_options.end()) {
+      return ADBC_STATUS_NOT_FOUND;
+    }
+    *value = it->second;
+    return ADBC_STATUS_OK;
+  }
+  return connection->private_driver->ConnectionGetOptionDouble(connection, key, value,
+                                                               error);
+}
+
 AdbcStatusCode AdbcConnectionGetTableSchema(struct AdbcConnection* connection,
                                             const char* catalog, const char* db_schema,
                                             const char* table_name,
@@ -423,6 +688,9 @@ AdbcStatusCode AdbcConnectionInit(struct AdbcConnection* connection,
   TempConnection* args = reinterpret_cast<TempConnection*>(connection->private_data);
   connection->private_data = nullptr;
   std::unordered_map<std::string, std::string> options = std::move(args->options);
+  std::unordered_map<std::string, int64_t> int_options = std::move(args->int_options);
+  std::unordered_map<std::string, double> double_options =
+      std::move(args->double_options);
   delete args;
 
   auto status = database->private_driver->ConnectionNew(connection, error);
@@ -434,6 +702,16 @@ AdbcStatusCode AdbcConnectionInit(struct AdbcConnection* connection,
         connection, option.first.c_str(), option.second.c_str(), error);
     if (status != ADBC_STATUS_OK) return status;
   }
+  for (const auto& option : int_options) {
+    status = database->private_driver->ConnectionSetOptionInt(
+        connection, option.first.c_str(), option.second, error);
+    if (status != ADBC_STATUS_OK) return status;
+  }
+  for (const auto& option : double_options) {
+    status = database->private_driver->ConnectionSetOptionDouble(
+        connection, option.first.c_str(), option.second, error);
+    if (status != ADBC_STATUS_OK) return status;
+  }
   return connection->private_driver->ConnectionInit(connection, database, error);
 }
 
@@ -498,6 +776,40 @@ AdbcStatusCode AdbcConnectionSetOption(struct AdbcConnection* connection, const
   return connection->private_driver->ConnectionSetOption(connection, key, value, error);
 }
 
+AdbcStatusCode AdbcConnectionSetOptionInt(struct AdbcConnection* connection,
+                                          const char* key, int64_t value,
+                                          struct AdbcError* error) {
+  if (!connection->private_data) {
+    SetError(error, "AdbcConnectionSetOptionInt: must AdbcConnectionNew first");
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  if (!connection->private_driver) {
+    // Init not yet called, save the option
+    TempConnection* args = reinterpret_cast<TempConnection*>(connection->private_data);
+    args->int_options[key] = value;
+    return ADBC_STATUS_OK;
+  }
+  return connection->private_driver->ConnectionSetOptionInt(connection, key, value,
+                                                            error);
+}
+
+AdbcStatusCode AdbcConnectionSetOptionDouble(struct AdbcConnection* connection,
+                                             const char* key, double value,
+                                             struct AdbcError* error) {
+  if (!connection->private_data) {
+    SetError(error, "AdbcConnectionSetOptionDouble: must AdbcConnectionNew first");
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  if (!connection->private_driver) {
+    // Init not yet called, save the option
+    TempConnection* args = reinterpret_cast<TempConnection*>(connection->private_data);
+    args->double_options[key] = value;
+    return ADBC_STATUS_OK;
+  }
+  return connection->private_driver->ConnectionSetOptionDouble(connection, key, value,
+                                                               error);
+}
+
 AdbcStatusCode AdbcStatementBind(struct AdbcStatement* statement,
                                  struct ArrowArray* values, struct ArrowSchema* schema,
                                  struct AdbcError* error) {
@@ -516,6 +828,14 @@ AdbcStatusCode AdbcStatementBindStream(struct AdbcStatement* statement,
   return statement->private_driver->StatementBindStream(statement, stream, error);
 }
 
+AdbcStatusCode AdbcStatementCancel(struct AdbcStatement* statement,
+                                   struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementCancel(statement, error);
+}
+
 // XXX: cpplint gets confused here if declared as 'struct ArrowSchema* schema'
 AdbcStatusCode AdbcStatementExecutePartitions(struct AdbcStatement* statement,
                                               ArrowSchema* schema,
@@ -540,6 +860,43 @@ AdbcStatusCode AdbcStatementExecuteQuery(struct AdbcStatement* statement,
                                                           error);
 }
 
+AdbcStatusCode AdbcStatementExecuteSchema(struct AdbcStatement* statement,
+                                          struct ArrowSchema* schema,
+                                          struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementExecuteSchema(statement, schema, error);
+}
+
+AdbcStatusCode AdbcStatementGetOption(struct AdbcStatement* statement, const char* key,
+                                      char* value, size_t* length,
+                                      struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementGetOption(statement, key, value, length,
+                                                       error);
+}
+
+AdbcStatusCode AdbcStatementGetOptionInt(struct AdbcStatement* statement, const char* key,
+                                         int64_t* value, struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementGetOptionInt(statement, key, value, error);
+}
+
+AdbcStatusCode AdbcStatementGetOptionDouble(struct AdbcStatement* statement,
+                                            const char* key, double* value,
+                                            struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementGetOptionDouble(statement, key, value,
+                                                             error);
+}
+
 AdbcStatusCode AdbcStatementGetParameterSchema(struct AdbcStatement* statement,
                                                struct ArrowSchema* schema,
                                                struct AdbcError* error) {
@@ -586,6 +943,24 @@ AdbcStatusCode AdbcStatementSetOption(struct AdbcStatement* statement, const cha
   return statement->private_driver->StatementSetOption(statement, key, value, error);
 }
 
+AdbcStatusCode AdbcStatementSetOptionInt(struct AdbcStatement* statement, const char* key,
+                                         int64_t value, struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementSetOptionInt(statement, key, value, error);
+}
+
+AdbcStatusCode AdbcStatementSetOptionDouble(struct AdbcStatement* statement,
+                                            const char* key, double value,
+                                            struct AdbcError* error) {
+  if (!statement->private_driver) {
+    return ADBC_STATUS_INVALID_STATE;
+  }
+  return statement->private_driver->StatementSetOptionDouble(statement, key, value,
+                                                             error);
+}
+
 AdbcStatusCode AdbcStatementSetSqlQuery(struct AdbcStatement* statement,
                                         const char* query, struct AdbcError* error) {
   if (!statement->private_driver) {
@@ -640,11 +1015,19 @@ AdbcStatusCode AdbcLoadDriver(const char* driver_name, const char* entrypoint,
   AdbcDriverInitFunc init_func;
   std::string error_message;
 
-  if (version != ADBC_VERSION_1_0_0) {
-    SetError(error, "Only ADBC 1.0.0 is supported");
-    return ADBC_STATUS_NOT_IMPLEMENTED;
+  switch (version) {
+    case ADBC_VERSION_1_0_0:
+    case ADBC_VERSION_1_1_0:
+      break;
+    default:
+      SetError(error, "Only ADBC 1.0.0 and 1.1.0 are supported");
+      return ADBC_STATUS_NOT_IMPLEMENTED;
   }
 
+  if (!raw_driver) {
+    SetError(error, "Must provide non-NULL raw_driver");
+    return ADBC_STATUS_INVALID_ARGUMENT;
+  }
   auto* driver = reinterpret_cast<struct AdbcDriver*>(raw_driver);
 
   if (!entrypoint) {
@@ -771,6 +1154,25 @@ AdbcStatusCode AdbcLoadDriver(const char* driver_name, const char* entrypoint,
 
 AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int version,
                                           void* raw_driver, struct AdbcError* error) {
+  constexpr std::array<int, 2> kSupportedVersions = {
+      ADBC_VERSION_1_1_0,
+      ADBC_VERSION_1_0_0,
+  };
+
+  if (!raw_driver) {
+    SetError(error, "Must provide non-NULL raw_driver");
+    return ADBC_STATUS_INVALID_ARGUMENT;
+  }
+
+  switch (version) {
+    case ADBC_VERSION_1_0_0:
+    case ADBC_VERSION_1_1_0:
+      break;
+    default:
+      SetError(error, "Only ADBC 1.0.0 and 1.1.0 are supported");
+      return ADBC_STATUS_NOT_IMPLEMENTED;
+  }
+
 #define FILL_DEFAULT(DRIVER, STUB) \
   if (!DRIVER->STUB) {             \
     DRIVER->STUB = &STUB;          \
@@ -781,12 +1183,20 @@ AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int vers
     return ADBC_STATUS_INTERNAL;                                               \
   }
 
-  auto result = init_func(version, raw_driver, error);
+  // Starting from the passed version, try each (older) version in
+  // succession with the underlying driver until we find one that's
+  // accepted.
+  AdbcStatusCode result = ADBC_STATUS_NOT_IMPLEMENTED;
+  for (const int try_version : kSupportedVersions) {
+    if (try_version > version) continue;
+    result = init_func(try_version, raw_driver, error);
+    if (result != ADBC_STATUS_NOT_IMPLEMENTED) break;
+  }
   if (result != ADBC_STATUS_OK) {
     return result;
   }
 
-  if (version == ADBC_VERSION_1_0_0) {
+  if (version >= ADBC_VERSION_1_0_0) {
     auto* driver = reinterpret_cast<struct AdbcDriver*>(raw_driver);
     CHECK_REQUIRED(driver, DatabaseNew);
     CHECK_REQUIRED(driver, DatabaseInit);
@@ -816,6 +1226,29 @@ AdbcStatusCode AdbcLoadDriverFromInitFunc(AdbcDriverInitFunc init_func, int vers
     FILL_DEFAULT(driver, StatementSetSqlQuery);
     FILL_DEFAULT(driver, StatementSetSubstraitPlan);
   }
+  if (version >= ADBC_VERSION_1_1_0) {
+    auto* driver = reinterpret_cast<struct AdbcDriver*>(raw_driver);
+    FILL_DEFAULT(driver, DatabaseGetOption);
+    FILL_DEFAULT(driver, DatabaseGetOptionInt);
+    FILL_DEFAULT(driver, DatabaseGetOptionDouble);
+    FILL_DEFAULT(driver, DatabaseSetOptionInt);
+    FILL_DEFAULT(driver, DatabaseSetOptionDouble);
+
+    FILL_DEFAULT(driver, ConnectionCancel);
+    FILL_DEFAULT(driver, ConnectionGetOption);
+    FILL_DEFAULT(driver, ConnectionGetOptionInt);
+    FILL_DEFAULT(driver, ConnectionGetOptionDouble);
+    FILL_DEFAULT(driver, ConnectionSetOptionInt);
+    FILL_DEFAULT(driver, ConnectionSetOptionDouble);
+
+    FILL_DEFAULT(driver, StatementCancel);
+    FILL_DEFAULT(driver, StatementExecuteSchema);
+    FILL_DEFAULT(driver, StatementGetOption);
+    FILL_DEFAULT(driver, StatementGetOptionInt);
+    FILL_DEFAULT(driver, StatementGetOptionDouble);
+    FILL_DEFAULT(driver, StatementSetOptionInt);
+    FILL_DEFAULT(driver, StatementSetOptionDouble);
+  }
 
   return ADBC_STATUS_OK;
 
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 d3c81191..73e90cec 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
@@ -28,6 +28,20 @@ import org.apache.arrow.vector.types.pojo.Schema;
  * long as clients take care to serialize accesses to a connection.
  */
 public interface AdbcConnection extends AutoCloseable, AdbcOptions {
+  /**
+   * Cancel execution of a query.
+   *
+   * <p>This can be used to interrupt execution of a method like {@link #getObjects(GetObjectsDepth,
+   * String, String, String, String[], String)}}.
+   *
+   * <p>This method must be thread-safe (other method are not necessarily thread-safe).
+   *
+   * @since ADBC API revision 1.1.0
+   */
+  default void cancel() throws AdbcException {
+    throw AdbcException.notImplemented("Statement does not support cancel");
+  }
+
   /** Commit the pending transaction. */
   default void commit() throws AdbcException {
     throw AdbcException.notImplemented("Connection does not support transactions");
@@ -102,7 +116,7 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
    *   <caption>The definition of the GetObjects result schema.</caption>
    * </table>
    *
-   * DB_SCHEMA_SCHEMA is a Struct with fields:
+   * <p>DB_SCHEMA_SCHEMA is a Struct with fields:
    *
    * <table border="1">
    *   <tr><th>Field Name</th>              <th>Field Type</th>             </tr>
@@ -111,7 +125,7 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
    *   <caption>The definition of DB_SCHEMA_SCHEMA.</caption>
    * </table>
    *
-   * TABLE_SCHEMA is a Struct with fields:
+   * <p>TABLE_SCHEMA is a Struct with fields:
    *
    * <table border="1">
    *   <tr><th>Field Name</th>              <th>Field Type</th>             </tr>
@@ -122,7 +136,7 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
    *   <caption>The definition of TABLE_SCHEMA.</caption>
    * </table>
    *
-   * COLUMN_SCHEMA is a Struct with fields:
+   * <p>COLUMN_SCHEMA is a Struct with fields:
    *
    * <table border="1">
    *   <tr><th>Field Name</th>              <th>Field Type</th>             <th>Comments</th></tr>
@@ -148,7 +162,7 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
    *   <caption>The definition of COLUMN_SCHEMA.</caption>
    * </table>
    *
-   * Notes:
+   * <p>Notes:
    *
    * <ol>
    *   <li>The column's ordinal position in the table (starting from 1).
@@ -157,7 +171,7 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
    *       provide JDBC/ODBC-compatible metadata in an agnostic manner.
    * </ol>
    *
-   * CONSTRAINT_SCHEMA is a Struct with fields:
+   * <p>CONSTRAINT_SCHEMA is a Struct with fields:
    *
    * <table border="1">
    *   <tr><th>Field Name</th>              <th>Field Type</th>             <th>Comments</th></tr>
@@ -174,7 +188,7 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
    *   <li>For FOREIGN KEY only, the referenced table and columns.
    * </ol>
    *
-   * USAGE_SCHEMA is a Struct with fields:
+   * <p>USAGE_SCHEMA is a Struct with fields:
    *
    * <table border="1">
    *   <tr><th>Field Name</th>              <th>Field Type</th>             </tr>
@@ -227,6 +241,83 @@ public interface AdbcConnection extends AutoCloseable, AdbcOptions {
     TABLES,
   }
 
+  /**
+   * Get statistics about the data distribution of table(s).
+   *
+   * <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>catalog_name</td>            <td>utf8</td>                   </tr>
+   *   <tr><td>catalog_db_schemas</td>      <td>list[DB_SCHEMA_SCHEMA]</td> </tr>
+   *   <caption>The definition of the GetStatistics result schema.</caption>
+   * </table>
+   *
+   * <p>DB_SCHEMA_SCHEMA is a Struct with fields:
+   *
+   * <table border="1">
+   *   <tr><th>Field Name</th>              <th>Field Type</th>             </tr>
+   *   <tr><td>db_schema_name</td>          <td>utf8</td>                   </tr>
+   *   <tr><td>db_schema_tables</td>        <td>list[TABLE_SCHEMA]</td>     </tr>
+   *   <caption>The definition of DB_SCHEMA_SCHEMA.</caption>
+   * </table>
+   *
+   * <p>STATISTICS_SCHEMA is a Struct with fields:
+   *
+   * <table border="1">
+   *   <tr><th>Field Name</th>              <th>Field Type</th>             <th>Comments</th></tr>
+   *   <tr><td>table_name</td>              <td>utf8 not null</td>          <td></td></tr>
+   *   <tr><td>column_name</td>             <td>utf8</td>                   <td>(1)</td></tr>
+   *   <tr><td>statistic_key</td>           <td>int16</td>                  <td>(2)</td></tr>
+   *   <tr><td>statistic_value</td>         <td>VALUE_SCHEMA not null</td>  <td></td></tr>
+   *   <tr><td>statistic_is_approximate</td><td>bool not null</td>          <td>(3)</td></tr>
+   *   <caption>The definition of STATISTICS_SCHEMA.</caption>
+   * </table>
+   *
+   * <ol>
+   *   <li>If null, then the statistic applies to the entire table.
+   *   <li>A dictionary-encoded statistic name (although we do not use the Arrow dictionary type).
+   *       Values in [0, 1024) are reserved for ADBC. Other values are for implementation-specific
+   *       statistics. For the definitions of predefined statistic types, see {@link
+   *       StandardStatistics}. To get driver-specific statistic names, use {@link
+   *       #getStatisticNames()}.
+   *   <li>If true, then the value is approximate or best-effort.
+   * </ol>
+   *
+   * <p>VALUE_SCHEMA is a dense union with members:
+   *
+   * <table border="1">
+   *   <tr><th>Field Name</th>              <th>Field Type</th>             </tr>
+   *   <tr><td>int64</td>                   <td>int64</td>                  </td></tr>
+   *   <tr><td>uint64</td>                  <td>uint64</td>                 </td></tr>
+   *   <tr><td>float64</td>                 <td>float64</td>                </td></tr>
+   *   <tr><td>decimal256</td>              <td>decimal256</td>             </td></tr>
+   *   <tr><td>binary</td>                  <td>binary</td>                 </td></tr>
+   *   <caption>The definition of VALUE_SCHEMA.</caption>
+   * </table>
+   *
+   * @param catalogPattern Only show tables in the given catalog. If null, do not filter by catalog.
+   *     If an empty string, only show tables without a catalog. May be a search pattern (see class
+   *     documentation).
+   * @param dbSchemaPattern Only show tables in the given database schema. If null, do not filter by
+   *     database schema. If an empty string, only show tables without a database schema. May be a
+   *     search pattern (see class documentation).
+   * @param tableNamePattern Only show tables with the given name. If an empty string, only show
+   *     tables without a catalog. May be a search pattern (see class documentation).
+   * @param approximate If false, request exact values of statistics, else allow for best-effort,
+   *     approximate, or cached values. The database may return approximate values regardless, as
+   *     indicated in the result. Requesting exact values may be expensive or unsupported.
+   */
+  default ArrowReader getStatistics(
+      String catalogPattern, String dbSchemaPattern, String tableNamePattern, boolean approximate)
+      throws AdbcException {
+    throw AdbcException.notImplemented("Connection does not support getStatistics()");
+  }
+
+  default ArrowReader getStatisticNames() throws AdbcException {
+    throw AdbcException.notImplemented("Connection does not support getStatisticNames()");
+  }
+
   /**
    * Get the Arrow schema of a database table.
    *
diff --git a/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcException.java b/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcException.java
index be5a4c6b..f909addf 100644
--- a/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcException.java
+++ b/java/core/src/main/java/org/apache/arrow/adbc/core/AdbcException.java
@@ -16,6 +16,10 @@
  */
 package org.apache.arrow.adbc.core;
 
+import java.nio.ByteBuffer;
+import java.util.Collection;
+import java.util.Collections;
+
 /**
  * An error in the database or ADBC driver.
  *
@@ -33,13 +37,25 @@ public class AdbcException extends Exception {
   private final AdbcStatusCode status;
   private final String sqlState;
   private final int vendorCode;
+  private Collection<ByteBuffer> details;
 
   public AdbcException(
       String message, Throwable cause, AdbcStatusCode status, String sqlState, int vendorCode) {
+    this(message, cause, status, sqlState, vendorCode, Collections.emptyList());
+  }
+
+  public AdbcException(
+      String message,
+      Throwable cause,
+      AdbcStatusCode status,
+      String sqlState,
+      int vendorCode,
+      Collection<ByteBuffer> details) {
     super(message, cause);
     this.status = status;
     this.sqlState = sqlState;
     this.vendorCode = vendorCode;
+    this.details = details;
   }
 
   /** Create a new exception with code {@link AdbcStatusCode#INVALID_ARGUMENT}. */
@@ -77,11 +93,30 @@ public class AdbcException extends Exception {
     return vendorCode;
   }
 
+  /**
+   * Get extra driver-specific binary error details.
+   *
+   * <p>This allows drivers to return custom, structured error information (for example, JSON or
+   * Protocol Buffers) that can be optionally parsed by clients, beyond the standard AdbcError
+   * fields, without having to encode it in the error message. The encoding of the data is
+   * driver-defined.
+   */
+  public Collection<ByteBuffer> getDetails() {
+    return details;
+  }
+
   /**
    * Copy this exception with a different cause (a convenience for use with the static factories).
    */
   public AdbcException withCause(Throwable cause) {
-    return new AdbcException(this.getMessage(), cause, status, sqlState, vendorCode);
+    return new AdbcException(getMessage(), cause, status, sqlState, vendorCode, details);
+  }
+
+  /**
+   * Copy this exception with different details (a convenience for use with the static factories).
+   */
+  public AdbcException withDetails(Collection<ByteBuffer> details) {
+    return new AdbcException(getMessage(), getCause(), status, sqlState, vendorCode, details);
   }
 
   @Override
@@ -98,6 +133,8 @@ public class AdbcException extends Exception {
         + vendorCode
         + ", cause="
         + getCause()
+        + ", details="
+        + getDetails().size()
         + '}';
   }
 }
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 a033726b..27708e1b 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
@@ -45,6 +45,8 @@ public interface AdbcStatement extends AutoCloseable, AdbcOptions {
   /**
    * Cancel execution of a query.
    *
+   * <p>This can be used to interrupt execution of a method like {@link #executeQuery()}.
+   *
    * <p>This method must be thread-safe (other method are not necessarily thread-safe).
    *
    * @since ADBC API revision 1.1.0
diff --git a/java/core/src/main/java/org/apache/arrow/adbc/core/StandardStatistics.java b/java/core/src/main/java/org/apache/arrow/adbc/core/StandardStatistics.java
new file mode 100644
index 00000000..5412c645
--- /dev/null
+++ b/java/core/src/main/java/org/apache/arrow/adbc/core/StandardStatistics.java
@@ -0,0 +1,81 @@
+/*
+ * 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.
+ */
+package org.apache.arrow.adbc.core;
+
+import java.util.Objects;
+
+/**
+ * Definitions of standard statistic names/keys.
+ *
+ * <p>Statistic names are returned from {@link AdbcConnection#getStatistics(String, String, String,
+ * boolean)} in a dictionary-encoded form. This class provides the names and dictionary-encoded form
+ * of statistics defined by ADBC.
+ */
+public enum StandardStatistics {
+  /**
+   * The average byte width statistic. The average size in bytes of a row in the column. Value type
+   * is float64.
+   *
+   * <p>For example, this is roughly the average length of a string for a string column.
+   */
+  AVERAGE_BYTE_WIDTH("adbc.statistic.byte_width", 0),
+  /**
+   * The distinct value count (NDV) statistic. The number of distinct values in the column. Value
+   * type is int64 (when not approximate) or float64 (when approximate).
+   */
+  DISTINCT_COUNT("adbc.statistic.distinct_count", 1),
+  /**
+   * The max byte width statistic. The maximum size in bytes of a row in the column. Value type is
+   * int64 (when not approximate) or float64 (when approximate).
+   *
+   * <p>For example, this is the maximum length of a string for a string column.
+   */
+  MAX_BYTE_WIDTH("adbc.statistic.byte_width", 2),
+  /** The max value statistic. Value type is column-dependent. */
+  MAX_VALUE_NAME("adbc.statistic.byte_width", 3),
+  /** The min value statistic. Value type is column-dependent. */
+  MIN_VALUE_NAME("adbc.statistic.byte_width", 4),
+  /**
+   * The null count statistic. The number of values that are null in the column. Value type is int64
+   * (when not approximate) or float64 (when approximate).
+   */
+  NULL_COUNT_NAME("adbc.statistic.null_count", 5),
+  /**
+   * The row count statistic. The number of rows in the column or table. Value type is int64 (when
+   * not approximate) or float64 (when approximate).
+   */
+  ROW_COUNT_NAME("adbc.statistic.row_count", 6),
+  ;
+
+  private final String name;
+  private final int key;
+
+  StandardStatistics(String name, int key) {
+    this.name = Objects.requireNonNull(name);
+    this.key = key;
+  }
+
+  /** Get the statistic name. */
+  public String getName() {
+    return name;
+  }
+
+  /** Get the dictionary-encoded name. */
+  public int getKey() {
+    return key;
+  }
+}