You are viewing a plain text version of this content. The canonical link for it is here.
Posted to github@arrow.apache.org by "mbrobbel (via GitHub)" <gi...@apache.org> on 2023/03/17 13:58:12 UTC

[GitHub] [arrow-adbc] mbrobbel commented on a diff in pull request #478: feat(rust): define the rust adbc api

mbrobbel commented on code in PR #478:
URL: https://github.com/apache/arrow-adbc/pull/478#discussion_r1140086272


##########
rust/.gitignore:
##########
@@ -0,0 +1,18 @@
+# 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.
+
+Cargo.lock

Review Comment:
   Does it make sense to add `/target` here, to prevent issues when the root `.gitignore` changes?
   ```suggestion
   /target
   Cargo.lock
   ```



##########
.github/workflows/rust.yml:
##########
@@ -0,0 +1,70 @@
+# 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.
+
+name: Rust
+
+on:
+  pull_request:
+    branches:
+      - main
+    paths:
+      - "rust/**"
+      - ".github/workflows/rust.yml"
+  push:
+    paths:
+      - "rust/**"
+      - ".github/workflows/rust.yml"
+
+concurrency:
+  group: ${{ github.repository }}-${{ github.ref }}-${{ github.workflow }}
+  cancel-in-progress: true
+
+permissions:
+  contents: read
+
+defaults:
+  run:
+    working-directory: rust
+
+jobs:
+  rust:
+    strategy:
+      matrix:
+        os: [windows-latest, macos-latest, ubuntu-latest]
+    name: "Rust ${{ matrix.os }}"
+    runs-on: ${{ matrix.os }}
+    steps:
+      - uses: actions/checkout@v3
+        with:
+          fetch-depth: 0
+          persist-credentials: false
+      - name: Install stable toolchain
+        uses: actions-rs/toolchain@v1
+        with:
+          profile: default
+          toolchain: stable
+          override: true

Review Comment:
   The `actions-rs` actions are unmaintained. I would recommend using:
   ```suggestion
         - name: Install stable toolchain
           uses: dtolnay/rust-toolchain@stable
           with:
             components: clippy, rustfmt
   ```



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_NAME: u32 = 0;
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_VERSION: u32 = 1;
+    /// The database vendor/product Arrow library version (type: utf8).
+    pub const VENDOR_ARROW_VERSION: u32 = 2;
+    /// The driver name (type: utf8).
+    pub const DRIVER_NAME: u32 = 100;
+    /// The driver version (type: utf8).
+    pub const DRIVER_VERSION: u32 = 101;
+    /// The driver Arrow library version (type: utf8).
+    pub const DRIVER_ARROW_VERSION: u32 = 102;
+}
+
+pub fn info_schema() -> Schema {
+    Schema::new(vec![
+        Field::new("info_name", DataType::UInt32, false),
+        Field::new(
+            "info_value",
+            DataType::Union(
+                vec![
+                    Field::new("string_value", DataType::Utf8, true),
+                    Field::new("bool_value", DataType::Boolean, true),
+                    Field::new("int64_value", DataType::Int64, true),
+                    Field::new("int32_bitmask", DataType::Int32, true),
+                    Field::new(
+                        "string_list",
+                        DataType::List(Box::new(Field::new("item", DataType::Utf8, true))),
+                        true,
+                    ),
+                    Field::new(
+                        "int32_to_int32_list_map",
+                        DataType::Map(
+                            Box::new(Field::new(
+                                "entries",
+                                DataType::Struct(vec![
+                                    Field::new("keys", DataType::Int32, false),
+                                    Field::new(
+                                        "values",
+                                        DataType::List(Box::new(Field::new(
+                                            "item",
+                                            DataType::Int32,
+                                            true,
+                                        ))),
+                                        true,
+                                    ),
+                                ]),
+                                false,
+                            )),
+                            false,
+                        ),
+                        true,
+                    ),
+                ],
+                vec![0, 1, 2, 3, 4, 5],
+                UnionMode::Dense,
+            ),
+            true,
+        ),
+    ])
+}
+
+/// Rust representations of database/drier metadata
+#[derive(Clone, Debug, PartialEq)]
+pub enum InfoData {
+    StringValue(Cow<'static, str>),
+    BoolValue(bool),
+    Int64Value(i64),
+    Int32Bitmask(i32),
+    StringList(Vec<String>),
+    Int32ToInt32ListMap(HashMap<i32, Vec<i32>>),
+}
+
+pub fn export_info_data(
+    info_iter: impl IntoIterator<Item = (u32, InfoData)>,
+) -> Box<dyn RecordBatchReader> {
+    let info_iter = info_iter.into_iter();
+
+    let mut codes = UInt32Builder::with_capacity(info_iter.size_hint().0);
+
+    // Type id tells which array the value is in
+    let mut type_id = UInt8BufferBuilder::new(info_iter.size_hint().0);
+    // Value offset tells the offset of the value in the respective array
+    let mut value_offsets = UInt32BufferBuilder::new(info_iter.size_hint().0);
+
+    // Make one builder per child of union array. Will combine after.
+    let mut string_values = StringBuilder::new();
+    let mut bool_values = BooleanBuilder::new();
+    let mut int64_values = Int64Builder::new();
+    let mut int32_bitmasks = Int32Builder::new();
+    let mut string_lists = ListBuilder::new(StringBuilder::new());
+    let mut int32_to_int32_list_maps = MapBuilder::new(
+        None,
+        Int32Builder::new(),
+        ListBuilder::new(Int32Builder::new()),
+    );
+
+    for (code, info) in info_iter {
+        codes.append_value(code);
+
+        match info {
+            InfoData::StringValue(val) => {
+                string_values.append_value(val);
+                type_id.append(0);
+                let value_offset = string_values.len() - 1;
+                value_offsets.append(
+                    value_offset
+                        .try_into()
+                        .expect("Array has more values than can be indexed by u32"),
+                );
+            }
+            _ => {
+                todo!("support other types in info_data")
+            }
+        };
+    }
+
+    let arrays: Vec<ArrayRef> = vec![
+        Arc::new(string_values.finish()),
+        Arc::new(bool_values.finish()),
+        Arc::new(int64_values.finish()),
+        Arc::new(int32_bitmasks.finish()),
+        Arc::new(string_lists.finish()),
+        Arc::new(int32_to_int32_list_maps.finish()),
+    ];
+    let info_schema = info_schema();
+    let union_fields = {
+        match info_schema.field(1).data_type() {
+            DataType::Union(fields, _, _) => fields,
+            _ => unreachable!(),
+        }
+    };
+    let children = union_fields
+        .iter()
+        .map(|f| f.to_owned())
+        .zip(arrays.into_iter())
+        .collect();
+    let info_value = UnionArray::try_new(
+        &[0, 1, 2, 3, 4, 5],
+        type_id.finish(),
+        Some(value_offsets.finish()),
+        children,
+    )
+    .expect("Info value array is always valid.");
+
+    let batch: RecordBatch = RecordBatch::try_new(
+        Arc::new(info_schema),
+        vec![Arc::new(codes.finish()), Arc::new(info_value)],
+    )
+    .expect("Info data batch is always valid.");
+
+    let schema = batch.schema();
+    Box::new(RecordBatchIterator::new(
+        std::iter::once(batch).map(Ok),
+        schema,
+    ))
+}
+
+pub fn import_info_data(
+    reader: Box<dyn RecordBatchReader>,
+) -> Result<Vec<(u32, InfoData)>, ArrowError> {
+    let batches = reader.collect::<Result<Vec<RecordBatch>, ArrowError>>()?;
+
+    Ok(batches
+        .iter()
+        .flat_map(|batch| {
+            let codes = as_primitive_array::<UInt32Type>(batch.column(0));
+            let codes = codes.into_iter().map(|code| code.unwrap());
+
+            let info_data = as_union_array(batch.column(1));
+            let info_data = (0..info_data.len()).map(|i| -> InfoData {
+                let type_id = info_data.type_id(i);
+                match type_id {
+                    0 => InfoData::StringValue(Cow::Owned(
+                        as_string_array(&info_data.value(i)).value(0).to_string(),
+                    )),
+                    _ => todo!("Support other types"),
+                }
+            });
+
+            std::iter::zip(codes, info_data)
+        })
+        .collect())
+}
+
+#[cfg(test)]
+mod test {
+    use std::ops::Deref;
+
+    use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+    use arrow_array::types::UInt32Type;
+
+    use super::*;
+
+    #[test]
+    fn test_export_info_data() {
+        let example_info = vec![
+            (
+                codes::VENDOR_NAME,
+                InfoData::StringValue(Cow::Borrowed("test vendor")),
+            ),
+            (
+                codes::DRIVER_NAME,
+                InfoData::StringValue(Cow::Borrowed("test driver")),
+            ),
+        ];
+
+        let info = export_info_data(example_info.clone());
+
+        assert_eq!(info.schema().deref(), &info_schema());
+        let info: HashMap<u32, String> = info
+            .flat_map(|maybe_batch| {
+                let batch = maybe_batch.unwrap();
+                let id = as_primitive_array::<UInt32Type>(batch.column(0));
+                let values = as_union_array(batch.column(1));
+                let string_values = as_string_array(values.child(0));
+                let mut out = vec![];
+                for i in 0..batch.num_rows() {
+                    assert_eq!(values.type_id(i), 0);
+                    out.push((id.value(i), string_values.value(i).to_string()));
+                }
+                out
+            })
+            .collect();
+
+        assert_eq!(
+            info.get(&codes::VENDOR_NAME),
+            Some(&"test vendor".to_string())
+        );
+        assert_eq!(
+            info.get(&codes::DRIVER_NAME),
+            Some(&"test driver".to_string())
+        );
+
+        let info = export_info_data(example_info);
+
+        let info: HashMap<u32, InfoData> = import_info_data(info).unwrap().into_iter().collect();
+        dbg!(&info);

Review Comment:
   ```suggestion
   ```



##########
rust/Cargo.toml:
##########
@@ -0,0 +1,41 @@
+# 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]
+name = "arrow-adbc"
+version = "0.1.0"
+edition = "2021"
+rust-version = "1.62"
+description = "Rust implementation of Arrow Database Connectivity (ADBC)"
+homepage = "https://arrow.apache.org/adbc/"
+repository = "https://github.com/apache/arrow-adbc"
+authors = ["Apache Arrow <de...@arrow.apache.org>"]
+license = "Apache-2.0"
+keywords = ["arrow", "database", "sql"]
+
+# See more keys and their definitions at https://doc.rust-lang.org/cargo/reference/manifest.html
+
+[dependencies]
+arrow-array = { version = "34.0.0", default_features = false}
+arrow-schema = { version = "34.0.0", features = ["ffi"], default_features = false}
+arrow-data = { version = "34.0.0", features = ["ffi"], default_features = false}

Review Comment:
   ```suggestion
   arrow-array = { version = "34.0.0", default-features = false}
   arrow-schema = { version = "34.0.0", features = ["ffi"], default-features = false}
   arrow-data = { version = "34.0.0", features = ["ffi"], default-features = false}
   ```



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(

Review Comment:
   Did you consider making this an `async` fn?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using

Review Comment:
   ```suggestion
   /// setting [options::ADBC_CONNECTION_OPTION_AUTOCOMMIT] to `"false"` (using
   ```



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {

Review Comment:
   Instead of `u32` const values could we use an enum here including an "other" variant that stores the code as `u32`? The well-defined variants should convert to `u32` when added to their array.



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;

Review Comment:
   Should we use an associated type for the return type?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;

Review Comment:
   `Vec<(u32, InfoData)>` suggests there could be duplicate values for the same key (`u32`), would a map be better?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;

Review Comment:
   How about making these key and values type generic and require them to convert to a `&str` or `String` via a trait bound to support strongly typed options?



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_NAME: u32 = 0;
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_VERSION: u32 = 1;
+    /// The database vendor/product Arrow library version (type: utf8).
+    pub const VENDOR_ARROW_VERSION: u32 = 2;
+    /// The driver name (type: utf8).
+    pub const DRIVER_NAME: u32 = 100;
+    /// The driver version (type: utf8).
+    pub const DRIVER_VERSION: u32 = 101;
+    /// The driver Arrow library version (type: utf8).
+    pub const DRIVER_ARROW_VERSION: u32 = 102;
+}
+
+pub fn info_schema() -> Schema {
+    Schema::new(vec![
+        Field::new("info_name", DataType::UInt32, false),
+        Field::new(
+            "info_value",
+            DataType::Union(
+                vec![
+                    Field::new("string_value", DataType::Utf8, true),
+                    Field::new("bool_value", DataType::Boolean, true),
+                    Field::new("int64_value", DataType::Int64, true),
+                    Field::new("int32_bitmask", DataType::Int32, true),
+                    Field::new(
+                        "string_list",
+                        DataType::List(Box::new(Field::new("item", DataType::Utf8, true))),
+                        true,
+                    ),
+                    Field::new(
+                        "int32_to_int32_list_map",
+                        DataType::Map(
+                            Box::new(Field::new(
+                                "entries",
+                                DataType::Struct(vec![
+                                    Field::new("keys", DataType::Int32, false),
+                                    Field::new(
+                                        "values",
+                                        DataType::List(Box::new(Field::new(
+                                            "item",
+                                            DataType::Int32,
+                                            true,
+                                        ))),
+                                        true,
+                                    ),
+                                ]),
+                                false,
+                            )),
+                            false,
+                        ),
+                        true,
+                    ),
+                ],
+                vec![0, 1, 2, 3, 4, 5],
+                UnionMode::Dense,
+            ),
+            true,
+        ),
+    ])
+}
+
+/// Rust representations of database/drier metadata
+#[derive(Clone, Debug, PartialEq)]
+pub enum InfoData {
+    StringValue(Cow<'static, str>),
+    BoolValue(bool),
+    Int64Value(i64),
+    Int32Bitmask(i32),
+    StringList(Vec<String>),
+    Int32ToInt32ListMap(HashMap<i32, Vec<i32>>),
+}
+
+pub fn export_info_data(
+    info_iter: impl IntoIterator<Item = (u32, InfoData)>,
+) -> Box<dyn RecordBatchReader> {

Review Comment:
   You can use `impl Trait` here:
   ```suggestion
   ) -> impl RecordBatchReader {
   ```



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_NAME: u32 = 0;
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_VERSION: u32 = 1;
+    /// The database vendor/product Arrow library version (type: utf8).
+    pub const VENDOR_ARROW_VERSION: u32 = 2;
+    /// The driver name (type: utf8).
+    pub const DRIVER_NAME: u32 = 100;
+    /// The driver version (type: utf8).
+    pub const DRIVER_VERSION: u32 = 101;
+    /// The driver Arrow library version (type: utf8).
+    pub const DRIVER_ARROW_VERSION: u32 = 102;
+}
+
+pub fn info_schema() -> Schema {
+    Schema::new(vec![
+        Field::new("info_name", DataType::UInt32, false),
+        Field::new(
+            "info_value",
+            DataType::Union(
+                vec![
+                    Field::new("string_value", DataType::Utf8, true),
+                    Field::new("bool_value", DataType::Boolean, true),
+                    Field::new("int64_value", DataType::Int64, true),
+                    Field::new("int32_bitmask", DataType::Int32, true),
+                    Field::new(
+                        "string_list",
+                        DataType::List(Box::new(Field::new("item", DataType::Utf8, true))),
+                        true,
+                    ),
+                    Field::new(
+                        "int32_to_int32_list_map",
+                        DataType::Map(
+                            Box::new(Field::new(
+                                "entries",
+                                DataType::Struct(vec![
+                                    Field::new("keys", DataType::Int32, false),
+                                    Field::new(
+                                        "values",
+                                        DataType::List(Box::new(Field::new(
+                                            "item",
+                                            DataType::Int32,
+                                            true,
+                                        ))),
+                                        true,
+                                    ),
+                                ]),
+                                false,
+                            )),
+                            false,
+                        ),
+                        true,
+                    ),
+                ],
+                vec![0, 1, 2, 3, 4, 5],
+                UnionMode::Dense,
+            ),
+            true,
+        ),
+    ])
+}

Review Comment:
   How about using something like [once_cell](https://docs.rs/once_cell)'s `Lazy` here?



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_NAME: u32 = 0;
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_VERSION: u32 = 1;
+    /// The database vendor/product Arrow library version (type: utf8).
+    pub const VENDOR_ARROW_VERSION: u32 = 2;
+    /// The driver name (type: utf8).
+    pub const DRIVER_NAME: u32 = 100;
+    /// The driver version (type: utf8).
+    pub const DRIVER_VERSION: u32 = 101;
+    /// The driver Arrow library version (type: utf8).
+    pub const DRIVER_ARROW_VERSION: u32 = 102;
+}
+
+pub fn info_schema() -> Schema {
+    Schema::new(vec![
+        Field::new("info_name", DataType::UInt32, false),
+        Field::new(
+            "info_value",
+            DataType::Union(
+                vec![
+                    Field::new("string_value", DataType::Utf8, true),
+                    Field::new("bool_value", DataType::Boolean, true),
+                    Field::new("int64_value", DataType::Int64, true),
+                    Field::new("int32_bitmask", DataType::Int32, true),
+                    Field::new(
+                        "string_list",
+                        DataType::List(Box::new(Field::new("item", DataType::Utf8, true))),
+                        true,
+                    ),
+                    Field::new(
+                        "int32_to_int32_list_map",
+                        DataType::Map(
+                            Box::new(Field::new(
+                                "entries",
+                                DataType::Struct(vec![
+                                    Field::new("keys", DataType::Int32, false),
+                                    Field::new(
+                                        "values",
+                                        DataType::List(Box::new(Field::new(
+                                            "item",
+                                            DataType::Int32,
+                                            true,
+                                        ))),
+                                        true,
+                                    ),
+                                ]),
+                                false,
+                            )),
+                            false,
+                        ),
+                        true,
+                    ),
+                ],
+                vec![0, 1, 2, 3, 4, 5],
+                UnionMode::Dense,
+            ),
+            true,
+        ),
+    ])
+}
+
+/// Rust representations of database/drier metadata
+#[derive(Clone, Debug, PartialEq)]
+pub enum InfoData {
+    StringValue(Cow<'static, str>),
+    BoolValue(bool),
+    Int64Value(i64),
+    Int32Bitmask(i32),
+    StringList(Vec<String>),
+    Int32ToInt32ListMap(HashMap<i32, Vec<i32>>),
+}
+
+pub fn export_info_data(
+    info_iter: impl IntoIterator<Item = (u32, InfoData)>,
+) -> Box<dyn RecordBatchReader> {
+    let info_iter = info_iter.into_iter();
+
+    let mut codes = UInt32Builder::with_capacity(info_iter.size_hint().0);
+
+    // Type id tells which array the value is in
+    let mut type_id = UInt8BufferBuilder::new(info_iter.size_hint().0);
+    // Value offset tells the offset of the value in the respective array
+    let mut value_offsets = UInt32BufferBuilder::new(info_iter.size_hint().0);
+
+    // Make one builder per child of union array. Will combine after.
+    let mut string_values = StringBuilder::new();
+    let mut bool_values = BooleanBuilder::new();
+    let mut int64_values = Int64Builder::new();
+    let mut int32_bitmasks = Int32Builder::new();
+    let mut string_lists = ListBuilder::new(StringBuilder::new());
+    let mut int32_to_int32_list_maps = MapBuilder::new(
+        None,
+        Int32Builder::new(),
+        ListBuilder::new(Int32Builder::new()),
+    );
+
+    for (code, info) in info_iter {
+        codes.append_value(code);
+
+        match info {
+            InfoData::StringValue(val) => {
+                string_values.append_value(val);
+                type_id.append(0);
+                let value_offset = string_values.len() - 1;
+                value_offsets.append(
+                    value_offset
+                        .try_into()
+                        .expect("Array has more values than can be indexed by u32"),
+                );
+            }
+            _ => {
+                todo!("support other types in info_data")
+            }
+        };
+    }
+
+    let arrays: Vec<ArrayRef> = vec![
+        Arc::new(string_values.finish()),
+        Arc::new(bool_values.finish()),
+        Arc::new(int64_values.finish()),
+        Arc::new(int32_bitmasks.finish()),
+        Arc::new(string_lists.finish()),
+        Arc::new(int32_to_int32_list_maps.finish()),
+    ];
+    let info_schema = info_schema();
+    let union_fields = {
+        match info_schema.field(1).data_type() {
+            DataType::Union(fields, _, _) => fields,
+            _ => unreachable!(),
+        }
+    };
+    let children = union_fields
+        .iter()
+        .map(|f| f.to_owned())
+        .zip(arrays.into_iter())
+        .collect();
+    let info_value = UnionArray::try_new(
+        &[0, 1, 2, 3, 4, 5],
+        type_id.finish(),
+        Some(value_offsets.finish()),
+        children,
+    )
+    .expect("Info value array is always valid.");
+
+    let batch: RecordBatch = RecordBatch::try_new(
+        Arc::new(info_schema),
+        vec![Arc::new(codes.finish()), Arc::new(info_value)],
+    )
+    .expect("Info data batch is always valid.");
+
+    let schema = batch.schema();
+    Box::new(RecordBatchIterator::new(
+        std::iter::once(batch).map(Ok),
+        schema,
+    ))

Review Comment:
   With the `impl Trait` return type this becomes:
   ```suggestion
       RecordBatchIterator::new(
           std::iter::once(batch).map(Ok),
           schema,
       )
   ```



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_NAME: u32 = 0;
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_VERSION: u32 = 1;
+    /// The database vendor/product Arrow library version (type: utf8).
+    pub const VENDOR_ARROW_VERSION: u32 = 2;
+    /// The driver name (type: utf8).
+    pub const DRIVER_NAME: u32 = 100;
+    /// The driver version (type: utf8).
+    pub const DRIVER_VERSION: u32 = 101;
+    /// The driver Arrow library version (type: utf8).
+    pub const DRIVER_ARROW_VERSION: u32 = 102;
+}
+
+pub fn info_schema() -> Schema {
+    Schema::new(vec![
+        Field::new("info_name", DataType::UInt32, false),
+        Field::new(
+            "info_value",
+            DataType::Union(
+                vec![
+                    Field::new("string_value", DataType::Utf8, true),
+                    Field::new("bool_value", DataType::Boolean, true),
+                    Field::new("int64_value", DataType::Int64, true),
+                    Field::new("int32_bitmask", DataType::Int32, true),
+                    Field::new(
+                        "string_list",
+                        DataType::List(Box::new(Field::new("item", DataType::Utf8, true))),
+                        true,
+                    ),
+                    Field::new(
+                        "int32_to_int32_list_map",
+                        DataType::Map(
+                            Box::new(Field::new(
+                                "entries",
+                                DataType::Struct(vec![
+                                    Field::new("keys", DataType::Int32, false),
+                                    Field::new(
+                                        "values",
+                                        DataType::List(Box::new(Field::new(
+                                            "item",
+                                            DataType::Int32,
+                                            true,
+                                        ))),
+                                        true,
+                                    ),
+                                ]),
+                                false,
+                            )),
+                            false,
+                        ),
+                        true,
+                    ),
+                ],
+                vec![0, 1, 2, 3, 4, 5],
+                UnionMode::Dense,
+            ),
+            true,
+        ),
+    ])
+}
+
+/// Rust representations of database/drier metadata
+#[derive(Clone, Debug, PartialEq)]
+pub enum InfoData {
+    StringValue(Cow<'static, str>),
+    BoolValue(bool),
+    Int64Value(i64),
+    Int32Bitmask(i32),
+    StringList(Vec<String>),
+    Int32ToInt32ListMap(HashMap<i32, Vec<i32>>),
+}
+
+pub fn export_info_data(
+    info_iter: impl IntoIterator<Item = (u32, InfoData)>,
+) -> Box<dyn RecordBatchReader> {
+    let info_iter = info_iter.into_iter();
+
+    let mut codes = UInt32Builder::with_capacity(info_iter.size_hint().0);
+
+    // Type id tells which array the value is in
+    let mut type_id = UInt8BufferBuilder::new(info_iter.size_hint().0);
+    // Value offset tells the offset of the value in the respective array
+    let mut value_offsets = UInt32BufferBuilder::new(info_iter.size_hint().0);
+
+    // Make one builder per child of union array. Will combine after.
+    let mut string_values = StringBuilder::new();
+    let mut bool_values = BooleanBuilder::new();
+    let mut int64_values = Int64Builder::new();
+    let mut int32_bitmasks = Int32Builder::new();
+    let mut string_lists = ListBuilder::new(StringBuilder::new());
+    let mut int32_to_int32_list_maps = MapBuilder::new(
+        None,
+        Int32Builder::new(),
+        ListBuilder::new(Int32Builder::new()),
+    );
+
+    for (code, info) in info_iter {
+        codes.append_value(code);
+
+        match info {
+            InfoData::StringValue(val) => {
+                string_values.append_value(val);
+                type_id.append(0);
+                let value_offset = string_values.len() - 1;
+                value_offsets.append(
+                    value_offset
+                        .try_into()
+                        .expect("Array has more values than can be indexed by u32"),
+                );
+            }
+            _ => {
+                todo!("support other types in info_data")
+            }
+        };
+    }
+
+    let arrays: Vec<ArrayRef> = vec![
+        Arc::new(string_values.finish()),
+        Arc::new(bool_values.finish()),
+        Arc::new(int64_values.finish()),
+        Arc::new(int32_bitmasks.finish()),
+        Arc::new(string_lists.finish()),
+        Arc::new(int32_to_int32_list_maps.finish()),
+    ];
+    let info_schema = info_schema();
+    let union_fields = {
+        match info_schema.field(1).data_type() {
+            DataType::Union(fields, _, _) => fields,
+            _ => unreachable!(),
+        }
+    };
+    let children = union_fields
+        .iter()
+        .map(|f| f.to_owned())
+        .zip(arrays.into_iter())
+        .collect();
+    let info_value = UnionArray::try_new(
+        &[0, 1, 2, 3, 4, 5],
+        type_id.finish(),
+        Some(value_offsets.finish()),
+        children,
+    )
+    .expect("Info value array is always valid.");
+
+    let batch: RecordBatch = RecordBatch::try_new(
+        Arc::new(info_schema),
+        vec![Arc::new(codes.finish()), Arc::new(info_value)],
+    )
+    .expect("Info data batch is always valid.");
+
+    let schema = batch.schema();
+    Box::new(RecordBatchIterator::new(
+        std::iter::once(batch).map(Ok),
+        schema,
+    ))
+}
+
+pub fn import_info_data(
+    reader: Box<dyn RecordBatchReader>,

Review Comment:
   Same here:
   ```suggestion
       reader: impl RecordBatchReader,
   ```



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;
+
+    /// Commit any pending transactions. Only used if autocommit is disabled.
+    fn commit(&self) -> Result<(), AdbcError>;
+
+    /// Roll back any pending transactions. Only used if autocommit is disabled.
+    fn rollback(&self) -> Result<(), AdbcError>;
+}
+
+/// Depth parameter for GetObjects method.
+#[derive(Debug)]
+#[repr(i32)]
+pub enum AdbcObjectDepth {
+    /// Metadata on catalogs, schemas, tables, and columns.
+    All = 0,
+    /// Metadata on catalogs only.
+    Catalogs = 1,
+    /// Metadata on catalogs and schemas.
+    DBSchemas = 2,
+    /// Metadata on catalogs, schemas, and tables.
+    Tables = 3,
+}
+
+/// A container for all state needed to execute a database query, such as the
+/// query itself, parameters for prepared statements, driver parameters, etc.
+///
+/// Statements may represent queries or prepared statements.
+///
+/// Statements may be used multiple times and can be reconfigured
+/// (e.g. they can be reused to execute multiple different queries).
+/// However, executing a statement (and changing certain other state)
+/// will invalidate result sets obtained prior to that execution.
+///
+/// Multiple statements may be created from a single connection.
+/// However, the driver may block or error if they are used
+/// concurrently (whether from a single thread or multiple threads).
+pub trait AdbcStatement {
+    /// Turn this statement into a prepared statement to be executed multiple time.
+    ///
+    /// This should return an error if called before [AdbcStatement::set_sql_query].
+    fn prepare(&mut self) -> Result<(), AdbcError>;
+
+    /// Set a string option on a statement.
+    fn set_option(&mut self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Set the SQL query to execute.
+    fn set_sql_query(&mut self, query: &str) -> Result<(), AdbcError>;
+
+    /// Set the Substrait plan to execute.
+    fn set_substrait_plan(&mut self, plan: &[u8]) -> Result<(), AdbcError>;

Review Comment:
   How about using the `Plan` type provided by the `substrait` crate here?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;

Review Comment:
   Same suggestion as above.



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;
+
+    /// Commit any pending transactions. Only used if autocommit is disabled.
+    fn commit(&self) -> Result<(), AdbcError>;
+
+    /// Roll back any pending transactions. Only used if autocommit is disabled.
+    fn rollback(&self) -> Result<(), AdbcError>;
+}
+
+/// Depth parameter for GetObjects method.
+#[derive(Debug)]

Review Comment:
   ```suggestion
   #[derive(Debug, Copy, Clone)]
   ```



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {

Review Comment:
   You can use lifetime elision with `'_`:
   ```suggestion
       fn get_catalog(&self, name: Option<&str>) -> Option<Self::CatalogEntryType<'_>> {
   ```



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;

Review Comment:
   Should this be an associated type?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;
+
+    /// Commit any pending transactions. Only used if autocommit is disabled.
+    fn commit(&self) -> Result<(), AdbcError>;
+
+    /// Roll back any pending transactions. Only used if autocommit is disabled.
+    fn rollback(&self) -> Result<(), AdbcError>;
+}
+
+/// Depth parameter for GetObjects method.
+#[derive(Debug)]
+#[repr(i32)]
+pub enum AdbcObjectDepth {
+    /// Metadata on catalogs, schemas, tables, and columns.
+    All = 0,
+    /// Metadata on catalogs only.
+    Catalogs = 1,
+    /// Metadata on catalogs and schemas.
+    DBSchemas = 2,
+    /// Metadata on catalogs, schemas, and tables.
+    Tables = 3,
+}
+
+/// A container for all state needed to execute a database query, such as the
+/// query itself, parameters for prepared statements, driver parameters, etc.
+///
+/// Statements may represent queries or prepared statements.
+///
+/// Statements may be used multiple times and can be reconfigured
+/// (e.g. they can be reused to execute multiple different queries).
+/// However, executing a statement (and changing certain other state)
+/// will invalidate result sets obtained prior to that execution.
+///
+/// Multiple statements may be created from a single connection.
+/// However, the driver may block or error if they are used
+/// concurrently (whether from a single thread or multiple threads).
+pub trait AdbcStatement {
+    /// Turn this statement into a prepared statement to be executed multiple time.
+    ///
+    /// This should return an error if called before [AdbcStatement::set_sql_query].
+    fn prepare(&mut self) -> Result<(), AdbcError>;
+
+    /// Set a string option on a statement.
+    fn set_option(&mut self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Set the SQL query to execute.
+    fn set_sql_query(&mut self, query: &str) -> Result<(), AdbcError>;
+
+    /// Set the Substrait plan to execute.
+    fn set_substrait_plan(&mut self, plan: &[u8]) -> Result<(), AdbcError>;
+
+    /// Get the schema for bound parameters.
+    ///
+    /// This retrieves an Arrow schema describing the number, names, and
+    /// types of the parameters in a parameterized statement.  The fields
+    /// of the schema should be in order of the ordinal position of the
+    /// parameters; named parameters should appear only once.
+    ///
+    /// If the parameter does not have a name, or the name cannot be
+    /// determined, the name of the corresponding field in the schema will
+    /// be an empty string.  If the type cannot be determined, the type of
+    /// the corresponding field will be NA (NullType).
+    ///
+    /// This should return an error if this was called before [AdbcStatement::prepare].
+    fn get_param_schema(&mut self) -> Result<Schema, AdbcError>;
+
+    /// Bind Arrow data, either for bulk inserts or prepared statements.
+    fn bind_data(&mut self, batch: RecordBatch) -> Result<(), AdbcError>;
+
+    /// Bind Arrow data, either for bulk inserts or prepared statements.
+    fn bind_stream(&mut self, stream: Box<dyn RecordBatchReader>) -> Result<(), AdbcError>;
+
+    /// Execute a statement and get the results.
+    ///
+    /// See [StatementResult].
+    fn execute(&mut self) -> Result<StatementResult, AdbcError>;
+
+    /// Execute a query that doesn't have a result set.
+    ///
+    /// Will return the number of rows affected, or -1 if unknown or unsupported.
+    fn execute_update(&mut self) -> Result<i64, AdbcError>;
+
+    /// Execute a statement with a partitioned result set.
+    ///
+    /// This is not required to be implemented, as it only applies to backends
+    /// that internally partition results. These backends can use this method
+    /// to support threaded or distributed clients.
+    ///
+    /// See [PartitionedStatementResult].
+    fn execute_partitioned(&mut self) -> Result<PartitionedStatementResult, AdbcError>;
+}
+
+/// Result of calling [AdbcStatement::execute].
+///
+/// `result` may be None if there is no meaningful result.
+/// `row_affected` may be -1 if not applicable or if it is not supported.
+pub struct StatementResult {
+    pub result: Option<Box<dyn RecordBatchReader>>,

Review Comment:
   Should this be using a generic instead of a trait object?



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;
+
+    /// Get a particular table by name.
+    ///
+    /// This is case sensitive
+    fn get_table(&self, name: &str) -> Option<Self::TableEntryType> {
+        self.tables().find(|table| table.name() == name)
+    }
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a single table.
+pub trait DatabaseTableEntry<'a> {
+    /// The name of the table.
+    fn name(&self) -> &'a str;
+
+    /// The table type.
+    ///
+    /// Use [crate::AdbcConnection::get_table_types] to get a list of supported types for
+    /// the database.
+    fn table_type(&self) -> &'a str;
+
+    /// List all the columns in the table.
+    fn columns(&self) -> Box<dyn Iterator<Item = ColumnSchemaRef<'a>> + 'a>;
+
+    /// Get a column for a particular ordinal position.
+    ///
+    /// Will return None if the column is not found.
+    fn get_column(&self, i: i32) -> Option<ColumnSchemaRef<'a>> {
+        self.columns().find(|col| col.ordinal_position == i)
+    }
+
+    /// Get a column by name.
+    ///
+    /// This is case sensitive. Will return None if the column is not found.
+    fn get_column_by_name(&self, name: &str) -> Option<ColumnSchemaRef<'a>> {

Review Comment:
   ```suggestion
       fn column_by_name(&self, name: &str) -> Option<ColumnSchemaRef<'a>> {
   ```



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;
+
+    /// Get a particular table by name.
+    ///
+    /// This is case sensitive
+    fn get_table(&self, name: &str) -> Option<Self::TableEntryType> {
+        self.tables().find(|table| table.name() == name)
+    }
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a single table.
+pub trait DatabaseTableEntry<'a> {
+    /// The name of the table.
+    fn name(&self) -> &'a str;
+
+    /// The table type.
+    ///
+    /// Use [crate::AdbcConnection::get_table_types] to get a list of supported types for
+    /// the database.
+    fn table_type(&self) -> &'a str;
+
+    /// List all the columns in the table.
+    fn columns(&self) -> Box<dyn Iterator<Item = ColumnSchemaRef<'a>> + 'a>;

Review Comment:
   Same question: associated type?



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;
+
+    /// Get a particular table by name.
+    ///
+    /// This is case sensitive

Review Comment:
   ```suggestion
       /// This is case sensitive.
   ```



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;

Review Comment:
   Should the return type be an associated type of this trait?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {

Review Comment:
   Should we also consider making some fns in this trait async?



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;
+
+    /// Get a particular table by name.
+    ///
+    /// This is case sensitive
+    fn get_table(&self, name: &str) -> Option<Self::TableEntryType> {
+        self.tables().find(|table| table.name() == name)
+    }
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a single table.
+pub trait DatabaseTableEntry<'a> {
+    /// The name of the table.
+    fn name(&self) -> &'a str;
+
+    /// The table type.
+    ///
+    /// Use [crate::AdbcConnection::get_table_types] to get a list of supported types for
+    /// the database.
+    fn table_type(&self) -> &'a str;
+
+    /// List all the columns in the table.
+    fn columns(&self) -> Box<dyn Iterator<Item = ColumnSchemaRef<'a>> + 'a>;
+
+    /// Get a column for a particular ordinal position.
+    ///
+    /// Will return None if the column is not found.
+    fn get_column(&self, i: i32) -> Option<ColumnSchemaRef<'a>> {
+        self.columns().find(|col| col.ordinal_position == i)
+    }
+
+    /// Get a column by name.
+    ///
+    /// This is case sensitive. Will return None if the column is not found.
+    fn get_column_by_name(&self, name: &str) -> Option<ColumnSchemaRef<'a>> {
+        self.columns().find(|col| col.name == name)
+    }
+
+    /// List all the constraints on the table.
+    fn constraints(&self) -> Box<dyn Iterator<Item = TableConstraintRef<'a>> + 'a>;
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a column.
+///
+/// `xdbc_` columns are provided for compatibility with ODBC/JDBC column metadata.
+pub struct ColumnSchemaRef<'a> {
+    /// The name of the column.
+    pub name: &'a str,
+    /// The ordinal position of the column.
+    pub ordinal_position: i32,
+    pub remarks: Option<&'a str>,
+    pub xdbc_data_type: Option<i16>,
+    pub xdbc_type_name: Option<&'a str>,
+    pub xdbc_column_size: Option<i32>,
+    pub xdbc_decimal_digits: Option<i16>,
+    pub xdbc_num_prec_radix: Option<i16>,
+    pub xdbc_nullable: Option<i16>,
+    pub xdbc_column_def: Option<&'a str>,
+    pub xdbc_sql_data_type: Option<i16>,
+    pub xdbc_datetime_sub: Option<i16>,
+    pub xdbc_char_octet_length: Option<i32>,
+    pub xdbc_is_nullable: Option<&'a str>,
+    pub xdbc_scope_catalog: Option<&'a str>,
+    pub xdbc_scope_schema: Option<&'a str>,
+    pub xdbc_scope_table: Option<&'a str>,
+    pub xdbc_is_autoincrement: Option<bool>,
+    pub xdbc_is_generatedcolumn: Option<bool>,
+}
+
+impl<'a> ColumnSchemaRef<'a> {
+    pub fn to_owned(&self) -> ColumnSchema {
+        ColumnSchema {
+            name: self.name.to_owned(),
+            ordinal_position: self.ordinal_position,
+            remarks: self.remarks.as_ref().map(|&s| s.to_owned()),
+            xdbc_data_type: self.xdbc_data_type,
+            xdbc_type_name: self.xdbc_type_name.as_ref().map(|&s| s.to_owned()),
+            xdbc_column_size: self.xdbc_column_size,
+            xdbc_decimal_digits: self.xdbc_decimal_digits,
+            xdbc_num_prec_radix: self.xdbc_num_prec_radix,
+            xdbc_nullable: self.xdbc_nullable,
+            xdbc_column_def: self.xdbc_column_def.as_ref().map(|&s| s.to_owned()),
+            xdbc_sql_data_type: self.xdbc_sql_data_type,
+            xdbc_datetime_sub: self.xdbc_datetime_sub,
+            xdbc_char_octet_length: self.xdbc_char_octet_length,
+            xdbc_is_nullable: self.xdbc_is_nullable.as_ref().map(|&s| s.to_owned()),
+            xdbc_scope_catalog: self.xdbc_scope_catalog.as_ref().map(|&s| s.to_owned()),
+            xdbc_scope_schema: self.xdbc_scope_schema.as_ref().map(|&s| s.to_owned()),
+            xdbc_scope_table: self.xdbc_scope_table.as_ref().map(|&s| s.to_owned()),
+            xdbc_is_autoincrement: self.xdbc_is_autoincrement,
+            xdbc_is_generatedcolumn: self.xdbc_is_generatedcolumn,
+        }
+    }
+}
+
+/// An owning version of [ColumnSchema].
+#[derive(Debug, Clone, Default, PartialEq)]
+pub struct ColumnSchema {
+    name: String,
+    ordinal_position: i32,
+    remarks: Option<String>,
+    xdbc_data_type: Option<i16>,
+    xdbc_type_name: Option<String>,
+    xdbc_column_size: Option<i32>,
+    xdbc_decimal_digits: Option<i16>,
+    xdbc_num_prec_radix: Option<i16>,
+    xdbc_nullable: Option<i16>,
+    xdbc_column_def: Option<String>,
+    xdbc_sql_data_type: Option<i16>,
+    xdbc_datetime_sub: Option<i16>,
+    xdbc_char_octet_length: Option<i32>,
+    xdbc_is_nullable: Option<String>,
+    xdbc_scope_catalog: Option<String>,
+    xdbc_scope_schema: Option<String>,
+    xdbc_scope_table: Option<String>,
+    xdbc_is_autoincrement: Option<bool>,
+    xdbc_is_generatedcolumn: Option<bool>,
+}
+
+impl ColumnSchema {
+    pub fn borrow(&self) -> ColumnSchemaRef<'_> {
+        ColumnSchemaRef {
+            name: &self.name,
+            ordinal_position: self.ordinal_position,
+            remarks: self.remarks.as_deref(),
+            xdbc_data_type: self.xdbc_data_type,
+            xdbc_type_name: self.xdbc_type_name.as_deref(),
+            xdbc_column_size: self.xdbc_column_size,
+            xdbc_decimal_digits: self.xdbc_decimal_digits,
+            xdbc_num_prec_radix: self.xdbc_num_prec_radix,
+            xdbc_nullable: self.xdbc_nullable,
+            xdbc_column_def: self.xdbc_column_def.as_deref(),
+            xdbc_sql_data_type: self.xdbc_sql_data_type,
+            xdbc_datetime_sub: self.xdbc_datetime_sub,
+            xdbc_char_octet_length: self.xdbc_char_octet_length,
+            xdbc_is_nullable: self.xdbc_is_nullable.as_deref(),
+            xdbc_scope_catalog: self.xdbc_scope_catalog.as_deref(),
+            xdbc_scope_schema: self.xdbc_scope_schema.as_deref(),
+            xdbc_scope_table: self.xdbc_scope_table.as_deref(),
+            xdbc_is_autoincrement: self.xdbc_is_autoincrement,
+            xdbc_is_generatedcolumn: self.xdbc_is_generatedcolumn,
+        }
+    }
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a table constraint.
+pub struct TableConstraintRef<'a> {
+    pub name: Option<&'a str>,
+    pub columns: Vec<&'a str>,
+    pub constraint_type: TableConstraintTypeRef<'a>,
+}
+
+pub enum TableConstraintTypeRef<'a> {

Review Comment:
   This item is missing a doc comment.



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;
+
+    /// Get a particular table by name.
+    ///
+    /// This is case sensitive
+    fn get_table(&self, name: &str) -> Option<Self::TableEntryType> {
+        self.tables().find(|table| table.name() == name)
+    }
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a single table.
+pub trait DatabaseTableEntry<'a> {
+    /// The name of the table.
+    fn name(&self) -> &'a str;
+
+    /// The table type.
+    ///
+    /// Use [crate::AdbcConnection::get_table_types] to get a list of supported types for
+    /// the database.
+    fn table_type(&self) -> &'a str;
+
+    /// List all the columns in the table.
+    fn columns(&self) -> Box<dyn Iterator<Item = ColumnSchemaRef<'a>> + 'a>;
+
+    /// Get a column for a particular ordinal position.
+    ///
+    /// Will return None if the column is not found.
+    fn get_column(&self, i: i32) -> Option<ColumnSchemaRef<'a>> {

Review Comment:
   I think it's more idiomatic to remove the `get_` prefix. (https://rust-lang.github.io/api-guidelines/naming.html#getter-names-follow-rust-convention-c-getter)
   ```suggestion
       fn column(&self, i: i32) -> Option<ColumnSchemaRef<'a>> {
   ```



##########
rust/src/info.rs:
##########
@@ -0,0 +1,287 @@
+// 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.
+
+//! Utilities for driver info
+//!
+//! For use with [crate::AdbcConnection::get_info].
+
+use arrow_array::builder::{
+    ArrayBuilder, BooleanBuilder, Int32Builder, Int64Builder, ListBuilder, MapBuilder,
+    StringBuilder, UInt32BufferBuilder, UInt32Builder, UInt8BufferBuilder,
+};
+use arrow_array::cast::{as_primitive_array, as_string_array, as_union_array};
+use arrow_array::types::UInt32Type;
+use arrow_array::{Array, ArrayRef, UnionArray};
+use arrow_array::{RecordBatch, RecordBatchIterator, RecordBatchReader};
+use arrow_schema::{ArrowError, DataType, Field, Schema, UnionMode};
+use std::{borrow::Cow, collections::HashMap, sync::Arc};
+
+/// Contains known info codes defined by ADBC.
+pub mod codes {
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_NAME: u32 = 0;
+    /// The database vendor/product version (type: utf8).
+    pub const VENDOR_VERSION: u32 = 1;
+    /// The database vendor/product Arrow library version (type: utf8).
+    pub const VENDOR_ARROW_VERSION: u32 = 2;
+    /// The driver name (type: utf8).
+    pub const DRIVER_NAME: u32 = 100;
+    /// The driver version (type: utf8).
+    pub const DRIVER_VERSION: u32 = 101;
+    /// The driver Arrow library version (type: utf8).
+    pub const DRIVER_ARROW_VERSION: u32 = 102;
+}
+
+pub fn info_schema() -> Schema {
+    Schema::new(vec![
+        Field::new("info_name", DataType::UInt32, false),
+        Field::new(
+            "info_value",
+            DataType::Union(
+                vec![
+                    Field::new("string_value", DataType::Utf8, true),
+                    Field::new("bool_value", DataType::Boolean, true),
+                    Field::new("int64_value", DataType::Int64, true),
+                    Field::new("int32_bitmask", DataType::Int32, true),
+                    Field::new(
+                        "string_list",
+                        DataType::List(Box::new(Field::new("item", DataType::Utf8, true))),
+                        true,
+                    ),
+                    Field::new(
+                        "int32_to_int32_list_map",
+                        DataType::Map(
+                            Box::new(Field::new(
+                                "entries",
+                                DataType::Struct(vec![
+                                    Field::new("keys", DataType::Int32, false),
+                                    Field::new(
+                                        "values",
+                                        DataType::List(Box::new(Field::new(
+                                            "item",
+                                            DataType::Int32,
+                                            true,
+                                        ))),
+                                        true,
+                                    ),
+                                ]),
+                                false,
+                            )),
+                            false,
+                        ),
+                        true,
+                    ),
+                ],
+                vec![0, 1, 2, 3, 4, 5],
+                UnionMode::Dense,
+            ),
+            true,
+        ),
+    ])
+}
+
+/// Rust representations of database/drier metadata
+#[derive(Clone, Debug, PartialEq)]
+pub enum InfoData {
+    StringValue(Cow<'static, str>),
+    BoolValue(bool),
+    Int64Value(i64),
+    Int32Bitmask(i32),
+    StringList(Vec<String>),
+    Int32ToInt32ListMap(HashMap<i32, Vec<i32>>),
+}
+
+pub fn export_info_data(
+    info_iter: impl IntoIterator<Item = (u32, InfoData)>,
+) -> Box<dyn RecordBatchReader> {
+    let info_iter = info_iter.into_iter();
+
+    let mut codes = UInt32Builder::with_capacity(info_iter.size_hint().0);
+
+    // Type id tells which array the value is in
+    let mut type_id = UInt8BufferBuilder::new(info_iter.size_hint().0);
+    // Value offset tells the offset of the value in the respective array
+    let mut value_offsets = UInt32BufferBuilder::new(info_iter.size_hint().0);
+
+    // Make one builder per child of union array. Will combine after.
+    let mut string_values = StringBuilder::new();
+    let mut bool_values = BooleanBuilder::new();
+    let mut int64_values = Int64Builder::new();
+    let mut int32_bitmasks = Int32Builder::new();
+    let mut string_lists = ListBuilder::new(StringBuilder::new());
+    let mut int32_to_int32_list_maps = MapBuilder::new(
+        None,
+        Int32Builder::new(),
+        ListBuilder::new(Int32Builder::new()),
+    );
+
+    for (code, info) in info_iter {
+        codes.append_value(code);
+
+        match info {
+            InfoData::StringValue(val) => {
+                string_values.append_value(val);
+                type_id.append(0);
+                let value_offset = string_values.len() - 1;
+                value_offsets.append(
+                    value_offset
+                        .try_into()
+                        .expect("Array has more values than can be indexed by u32"),
+                );
+            }
+            _ => {
+                todo!("support other types in info_data")
+            }
+        };
+    }
+
+    let arrays: Vec<ArrayRef> = vec![
+        Arc::new(string_values.finish()),
+        Arc::new(bool_values.finish()),
+        Arc::new(int64_values.finish()),
+        Arc::new(int32_bitmasks.finish()),
+        Arc::new(string_lists.finish()),
+        Arc::new(int32_to_int32_list_maps.finish()),
+    ];
+    let info_schema = info_schema();
+    let union_fields = {
+        match info_schema.field(1).data_type() {
+            DataType::Union(fields, _, _) => fields,
+            _ => unreachable!(),
+        }
+    };
+    let children = union_fields
+        .iter()
+        .map(|f| f.to_owned())
+        .zip(arrays.into_iter())
+        .collect();
+    let info_value = UnionArray::try_new(
+        &[0, 1, 2, 3, 4, 5],
+        type_id.finish(),
+        Some(value_offsets.finish()),
+        children,
+    )
+    .expect("Info value array is always valid.");
+
+    let batch: RecordBatch = RecordBatch::try_new(
+        Arc::new(info_schema),
+        vec![Arc::new(codes.finish()), Arc::new(info_value)],
+    )
+    .expect("Info data batch is always valid.");
+
+    let schema = batch.schema();
+    Box::new(RecordBatchIterator::new(
+        std::iter::once(batch).map(Ok),
+        schema,
+    ))
+}
+
+pub fn import_info_data(
+    reader: Box<dyn RecordBatchReader>,
+) -> Result<Vec<(u32, InfoData)>, ArrowError> {
+    let batches = reader.collect::<Result<Vec<RecordBatch>, ArrowError>>()?;
+
+    Ok(batches
+        .iter()
+        .flat_map(|batch| {

Review Comment:
   You could use https://docs.rs/itertools/latest/itertools/trait.Itertools.html#method.flatten_ok here to prevent the early collect?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;

Review Comment:
   How about making different functions for this (ignore the fn names):
   ```rust
   fn get_info(&self, info_code: u32) -> Result<(u32, InfoData)>, AdbcError>;
   fn get_infos(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>>;
   ```



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;

Review Comment:
   Should the return type be an associated type?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;
+
+    /// Commit any pending transactions. Only used if autocommit is disabled.
+    fn commit(&self) -> Result<(), AdbcError>;
+
+    /// Roll back any pending transactions. Only used if autocommit is disabled.
+    fn rollback(&self) -> Result<(), AdbcError>;
+}
+
+/// Depth parameter for GetObjects method.
+#[derive(Debug)]
+#[repr(i32)]
+pub enum AdbcObjectDepth {
+    /// Metadata on catalogs, schemas, tables, and columns.
+    All = 0,
+    /// Metadata on catalogs only.
+    Catalogs = 1,
+    /// Metadata on catalogs and schemas.
+    DBSchemas = 2,
+    /// Metadata on catalogs, schemas, and tables.
+    Tables = 3,
+}
+
+/// A container for all state needed to execute a database query, such as the
+/// query itself, parameters for prepared statements, driver parameters, etc.
+///
+/// Statements may represent queries or prepared statements.
+///
+/// Statements may be used multiple times and can be reconfigured
+/// (e.g. they can be reused to execute multiple different queries).
+/// However, executing a statement (and changing certain other state)
+/// will invalidate result sets obtained prior to that execution.
+///
+/// Multiple statements may be created from a single connection.
+/// However, the driver may block or error if they are used
+/// concurrently (whether from a single thread or multiple threads).
+pub trait AdbcStatement {
+    /// Turn this statement into a prepared statement to be executed multiple time.
+    ///
+    /// This should return an error if called before [AdbcStatement::set_sql_query].
+    fn prepare(&mut self) -> Result<(), AdbcError>;
+
+    /// Set a string option on a statement.
+    fn set_option(&mut self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Set the SQL query to execute.
+    fn set_sql_query(&mut self, query: &str) -> Result<(), AdbcError>;
+
+    /// Set the Substrait plan to execute.
+    fn set_substrait_plan(&mut self, plan: &[u8]) -> Result<(), AdbcError>;
+
+    /// Get the schema for bound parameters.
+    ///
+    /// This retrieves an Arrow schema describing the number, names, and
+    /// types of the parameters in a parameterized statement.  The fields
+    /// of the schema should be in order of the ordinal position of the
+    /// parameters; named parameters should appear only once.
+    ///
+    /// If the parameter does not have a name, or the name cannot be
+    /// determined, the name of the corresponding field in the schema will
+    /// be an empty string.  If the type cannot be determined, the type of
+    /// the corresponding field will be NA (NullType).
+    ///
+    /// This should return an error if this was called before [AdbcStatement::prepare].
+    fn get_param_schema(&mut self) -> Result<Schema, AdbcError>;
+
+    /// Bind Arrow data, either for bulk inserts or prepared statements.
+    fn bind_data(&mut self, batch: RecordBatch) -> Result<(), AdbcError>;
+
+    /// Bind Arrow data, either for bulk inserts or prepared statements.
+    fn bind_stream(&mut self, stream: Box<dyn RecordBatchReader>) -> Result<(), AdbcError>;
+
+    /// Execute a statement and get the results.
+    ///
+    /// See [StatementResult].
+    fn execute(&mut self) -> Result<StatementResult, AdbcError>;
+
+    /// Execute a query that doesn't have a result set.
+    ///
+    /// Will return the number of rows affected, or -1 if unknown or unsupported.

Review Comment:
   Just to confirm: `Ok(-1)` means the query succeeded?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;
+
+    /// Commit any pending transactions. Only used if autocommit is disabled.
+    fn commit(&self) -> Result<(), AdbcError>;
+
+    /// Roll back any pending transactions. Only used if autocommit is disabled.
+    fn rollback(&self) -> Result<(), AdbcError>;
+}
+
+/// Depth parameter for GetObjects method.
+#[derive(Debug)]
+#[repr(i32)]
+pub enum AdbcObjectDepth {
+    /// Metadata on catalogs, schemas, tables, and columns.
+    All = 0,
+    /// Metadata on catalogs only.
+    Catalogs = 1,
+    /// Metadata on catalogs and schemas.
+    DBSchemas = 2,
+    /// Metadata on catalogs, schemas, and tables.
+    Tables = 3,
+}
+
+/// A container for all state needed to execute a database query, such as the
+/// query itself, parameters for prepared statements, driver parameters, etc.
+///
+/// Statements may represent queries or prepared statements.
+///
+/// Statements may be used multiple times and can be reconfigured
+/// (e.g. they can be reused to execute multiple different queries).
+/// However, executing a statement (and changing certain other state)
+/// will invalidate result sets obtained prior to that execution.
+///
+/// Multiple statements may be created from a single connection.
+/// However, the driver may block or error if they are used
+/// concurrently (whether from a single thread or multiple threads).
+pub trait AdbcStatement {
+    /// Turn this statement into a prepared statement to be executed multiple time.
+    ///
+    /// This should return an error if called before [AdbcStatement::set_sql_query].
+    fn prepare(&mut self) -> Result<(), AdbcError>;
+
+    /// Set a string option on a statement.
+    fn set_option(&mut self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Set the SQL query to execute.
+    fn set_sql_query(&mut self, query: &str) -> Result<(), AdbcError>;
+
+    /// Set the Substrait plan to execute.
+    fn set_substrait_plan(&mut self, plan: &[u8]) -> Result<(), AdbcError>;
+
+    /// Get the schema for bound parameters.
+    ///
+    /// This retrieves an Arrow schema describing the number, names, and
+    /// types of the parameters in a parameterized statement.  The fields
+    /// of the schema should be in order of the ordinal position of the
+    /// parameters; named parameters should appear only once.
+    ///
+    /// If the parameter does not have a name, or the name cannot be
+    /// determined, the name of the corresponding field in the schema will
+    /// be an empty string.  If the type cannot be determined, the type of
+    /// the corresponding field will be NA (NullType).
+    ///
+    /// This should return an error if this was called before [AdbcStatement::prepare].
+    fn get_param_schema(&mut self) -> Result<Schema, AdbcError>;
+
+    /// Bind Arrow data, either for bulk inserts or prepared statements.
+    fn bind_data(&mut self, batch: RecordBatch) -> Result<(), AdbcError>;
+
+    /// Bind Arrow data, either for bulk inserts or prepared statements.
+    fn bind_stream(&mut self, stream: Box<dyn RecordBatchReader>) -> Result<(), AdbcError>;

Review Comment:
   Should the stream be a generic type that implements `RecordBatchReader` instead?



##########
rust/src/lib.rs:
##########
@@ -0,0 +1,380 @@
+// 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.
+
+//! Arrow Database Connectivity (ADBC) allows efficient connections to databases
+//! for OLAP workloads:
+//!
+//!  * Uses the Arrow [C Data interface](https://arrow.apache.org/docs/format/CDataInterface.html)
+//!    and [C Stream Interface](https://arrow.apache.org/docs/format/CStreamInterface.html)
+//!    for efficient data interchange.
+//!  * Supports partitioned result sets for multi-threaded or distributed
+//!    applications.
+//!  * Support for [Substrait](https://substrait.io/) plans in addition to SQL queries.
+//!
+//! When implemented for remote databases, [Flight SQL](https://arrow.apache.org/docs/format/FlightSql.html)
+//! can be used as the communication protocol. This means data can be in Arrow
+//! format through the whole connection, minimizing serialization and deserialization
+//! overhead.
+//!
+//! Read more about ADBC at <https://arrow.apache.org/adbc/>
+//!
+//! There are two flavors of ADBC that this library supports:
+//!
+//!  * **Native Rust implementations**. These implement the traits at the top level of
+//!    this crate, starting with [AdbcDatabase].
+//!  * **C API ADBC drivers**. These can be implemented in any language (that compiles
+//!    to native code) and can be used by any language.
+//!
+//! # Native Rust drivers
+//!
+//! Native Rust drivers will implement the traits:
+//!
+//!  * [AdbcDatabase]
+//!  * [AdbcConnection]
+//!  * [AdbcStatement]
+//!
+//! For drivers implemented in Rust, using these will be more efficient and safe,
+//! since it avoids the overhead of going through C FFI.
+//!
+//! # Using C API drivers
+//!
+//! 🚧 TODO
+//!
+//! # Creating C API drivers
+//!
+//! 🚧 TODO
+//!
+pub mod error;
+pub mod info;
+pub mod objects;
+
+use arrow_array::{RecordBatch, RecordBatchReader};
+use arrow_schema::Schema;
+
+use crate::error::AdbcError;
+use crate::info::InfoData;
+
+/// Databases hold state shared by multiple connections. This typically means
+/// configuration and caches. For in-memory databases, it provides a place to
+/// hold ownership of the in-memory database.
+pub trait AdbcDatabase {
+    type ConnectionType: AdbcConnection;
+
+    /// Set an option on the database.
+    ///
+    /// Some databases may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Initialize a connection to the database.
+    ///
+    /// `options` provided will configure the connection, including the isolation
+    /// level. See standard options in [options].
+    fn connect<K, V>(
+        &self,
+        options: impl IntoIterator<Item = (K, V)>,
+    ) -> Result<Self::ConnectionType, AdbcError>
+    where
+        K: AsRef<str>,
+        V: AsRef<str>;
+}
+
+/// A connection is a single connection to a database.
+///
+/// It is never accessed concurrently from multiple threads.
+///
+/// # Autocommit
+///
+/// Connections should start in autocommit mode. They can be moved out by
+/// setting `"adbc.connection.autocommit"` to `"false"` (using
+/// [AdbcConnection::set_option]). Turning off autocommit allows customizing
+/// the isolation level. Read more in [adbc.h](https://github.com/apache/arrow-adbc/blob/main/adbc.h).
+pub trait AdbcConnection {
+    type StatementType: AdbcStatement;
+    type ObjectCollectionType: objects::DatabaseCatalogCollection;
+
+    /// Set an option on the connection.
+    ///
+    /// Some connections may not allow setting options after it has been initialized.
+    fn set_option(&self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Create a new [AdbcStatement].
+    fn new_statement(&self) -> Result<Self::StatementType, AdbcError>;
+
+    /// Get metadata about the database/driver.
+    ///
+    /// If None is passed for `info_codes`, the method will return all info.
+    /// Otherwise will return the specified info, in any order. If an unrecognized
+    /// code is passed, it will return an error.
+    ///
+    /// Each metadatum is identified by an integer code.  The recognized
+    /// codes are defined as constants.  Codes [0, 10_000) are reserved
+    /// for ADBC usage.  Drivers/vendors will ignore requests for
+    /// unrecognized codes (the row will be omitted from the result).
+    /// Known codes are provided in [info::codes].
+    fn get_info(&self, info_codes: Option<&[u32]>) -> Result<Vec<(u32, InfoData)>, AdbcError>;
+
+    /// Get a hierarchical view of all catalogs, database schemas, tables, and columns.
+    ///
+    /// # Parameters
+    ///
+    /// * **depth**: The level of nesting to display. If [AdbcObjectDepth::All], display
+    ///   all levels. If [AdbcObjectDepth::Catalogs], display only catalogs (i.e.  `catalog_schemas`
+    ///   will be null). If [AdbcObjectDepth::DBSchemas], display only catalogs and schemas
+    ///   (i.e. `db_schema_tables` will be null), and so on.
+    /// * **catalog**: Only show tables in the given catalog. If None,
+    ///   do not filter by catalog. If an empty string, only show tables
+    ///   without a catalog.  May be a search pattern (see next section).
+    /// * **db_schema**: Only show tables in the given database schema. If
+    ///   None, do not filter by database schema. If an empty string, only show
+    ///   tables without a database schema. May be a search pattern (see next section).
+    /// * **table_name**: Only show tables with the given name. If None, do not
+    ///   filter by name. May be a search pattern (see next section).
+    /// * **table_type**: Only show tables matching one of the given table
+    ///   types. If None, show tables of any type. Valid table types should
+    ///   match those returned by [AdbcConnection::get_table_schema].
+    /// * **column_name**: Only show columns with the given name. If
+    ///   None, do not filter by name.  May be a search pattern (see next section).
+    ///
+    /// # Search patterns
+    ///
+    /// Some parameters accept "search patterns", which are
+    /// strings that can contain the special character `"%"` to match zero
+    /// or more characters, or `"_"` to match exactly one character.  (See
+    /// the documentation of DatabaseMetaData in JDBC or "Pattern Value
+    /// Arguments" in the ODBC documentation.)
+    fn get_objects(
+        &self,
+        depth: AdbcObjectDepth,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: Option<&str>,
+        table_type: Option<&[&str]>,
+        column_name: Option<&str>,
+    ) -> Result<Self::ObjectCollectionType, AdbcError>;
+
+    /// Get the Arrow schema of a table.
+    ///
+    /// `catalog` or `db_schema` may be `None` when not applicable.
+    fn get_table_schema(
+        &self,
+        catalog: Option<&str>,
+        db_schema: Option<&str>,
+        table_name: &str,
+    ) -> Result<Schema, AdbcError>;
+
+    /// Get a list of table types in the database.
+    ///
+    /// The result is an Arrow dataset with the following schema:
+    ///
+    /// Field Name       | Field Type
+    /// -----------------|--------------
+    /// `table_type`     | `utf8 not null`
+    fn get_table_types(&self) -> Result<Vec<String>, AdbcError>;
+
+    /// Read part of a partitioned result set.
+    fn read_partition(&self, partition: &[u8]) -> Result<Box<dyn RecordBatchReader>, AdbcError>;
+
+    /// Commit any pending transactions. Only used if autocommit is disabled.
+    fn commit(&self) -> Result<(), AdbcError>;
+
+    /// Roll back any pending transactions. Only used if autocommit is disabled.
+    fn rollback(&self) -> Result<(), AdbcError>;
+}
+
+/// Depth parameter for GetObjects method.
+#[derive(Debug)]
+#[repr(i32)]
+pub enum AdbcObjectDepth {
+    /// Metadata on catalogs, schemas, tables, and columns.
+    All = 0,
+    /// Metadata on catalogs only.
+    Catalogs = 1,
+    /// Metadata on catalogs and schemas.
+    DBSchemas = 2,
+    /// Metadata on catalogs, schemas, and tables.
+    Tables = 3,
+}
+
+/// A container for all state needed to execute a database query, such as the
+/// query itself, parameters for prepared statements, driver parameters, etc.
+///
+/// Statements may represent queries or prepared statements.
+///
+/// Statements may be used multiple times and can be reconfigured
+/// (e.g. they can be reused to execute multiple different queries).
+/// However, executing a statement (and changing certain other state)
+/// will invalidate result sets obtained prior to that execution.
+///
+/// Multiple statements may be created from a single connection.
+/// However, the driver may block or error if they are used
+/// concurrently (whether from a single thread or multiple threads).
+pub trait AdbcStatement {
+    /// Turn this statement into a prepared statement to be executed multiple time.
+    ///
+    /// This should return an error if called before [AdbcStatement::set_sql_query].
+    fn prepare(&mut self) -> Result<(), AdbcError>;
+
+    /// Set a string option on a statement.
+    fn set_option(&mut self, key: &str, value: &str) -> Result<(), AdbcError>;
+
+    /// Set the SQL query to execute.
+    fn set_sql_query(&mut self, query: &str) -> Result<(), AdbcError>;
+
+    /// Set the Substrait plan to execute.
+    fn set_substrait_plan(&mut self, plan: &[u8]) -> Result<(), AdbcError>;
+
+    /// Get the schema for bound parameters.
+    ///
+    /// This retrieves an Arrow schema describing the number, names, and
+    /// types of the parameters in a parameterized statement.  The fields
+    /// of the schema should be in order of the ordinal position of the
+    /// parameters; named parameters should appear only once.
+    ///
+    /// If the parameter does not have a name, or the name cannot be
+    /// determined, the name of the corresponding field in the schema will
+    /// be an empty string.  If the type cannot be determined, the type of
+    /// the corresponding field will be NA (NullType).
+    ///
+    /// This should return an error if this was called before [AdbcStatement::prepare].
+    fn get_param_schema(&mut self) -> Result<Schema, AdbcError>;

Review Comment:
   Does this need `&mut self`?



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;
+
+    /// Get a particular catalog by name.
+    ///
+    /// Databases that have no notion of catalogs will have one with None for a name.
+    /// This is case sensitive.
+    fn get_catalog<'a>(&'a self, name: Option<&str>) -> Option<Self::CatalogEntryType<'a>> {
+        self.catalogs().find(|catalog| catalog.name() == name)
+    }
+}
+
+/// An entry in a [DatabaseCatalogCollection] representing a single catalog.
+pub trait DatabaseCatalogEntry<'a> {
+    type SchemaEntryType: DatabaseSchemaEntry<'a> + 'a;
+
+    /// Get the name of the catalog.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all schemas in this catalog that are in the result set.
+    fn schemas(&self) -> Box<dyn Iterator<Item = Self::SchemaEntryType> + 'a>;
+
+    /// Get a particular schema by name.
+    ///
+    /// Databases that have no notion of schemas will have one with None for a name.
+    /// This is case sensitive.
+    fn get_schema(&self, name: Option<&str>) -> Option<Self::SchemaEntryType> {
+        self.schemas().find(|schema| schema.name() == name)
+    }
+}
+
+/// An entry in [DatabaseCatalogCollection] representing a single schema.
+pub trait DatabaseSchemaEntry<'a> {
+    type TableEntryType: DatabaseTableEntry<'a>;
+
+    /// Get the name of the schema.
+    fn name(&self) -> Option<&'a str>;
+
+    /// List all the tables in this schema that are in the result set.
+    fn tables(&self) -> Box<dyn Iterator<Item = Self::TableEntryType> + 'a>;
+
+    /// Get a particular table by name.
+    ///
+    /// This is case sensitive
+    fn get_table(&self, name: &str) -> Option<Self::TableEntryType> {
+        self.tables().find(|table| table.name() == name)
+    }
+}
+
+/// An entry in the [DatabaseCatalogCollection] representing a single table.
+pub trait DatabaseTableEntry<'a> {
+    /// The name of the table.
+    fn name(&self) -> &'a str;
+
+    /// The table type.
+    ///
+    /// Use [crate::AdbcConnection::get_table_types] to get a list of supported types for
+    /// the database.
+    fn table_type(&self) -> &'a str;
+
+    /// List all the columns in the table.
+    fn columns(&self) -> Box<dyn Iterator<Item = ColumnSchemaRef<'a>> + 'a>;
+
+    /// Get a column for a particular ordinal position.
+    ///
+    /// Will return None if the column is not found.
+    fn get_column(&self, i: i32) -> Option<ColumnSchemaRef<'a>> {
+        self.columns().find(|col| col.ordinal_position == i)
+    }
+
+    /// Get a column by name.
+    ///
+    /// This is case sensitive. Will return None if the column is not found.
+    fn get_column_by_name(&self, name: &str) -> Option<ColumnSchemaRef<'a>> {
+        self.columns().find(|col| col.name == name)
+    }
+
+    /// List all the constraints on the table.
+    fn constraints(&self) -> Box<dyn Iterator<Item = TableConstraintRef<'a>> + 'a>;

Review Comment:
   Associated type?



##########
rust/src/objects.rs:
##########
@@ -0,0 +1,483 @@
+// 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.
+
+//! Structs and traits for representing database objects (tables, columns, schemas).
+//!
+//! When [crate::AdbcConnection::get_objects] is called, it returns an associated type that
+//! implements [DatabaseCatalogCollection]. This collection contains a hierarchical data
+//! structure representing:
+//!
+//!  * Database catalogs
+//!  * Database schemas
+//!  * Tables
+//!  * Columns
+//!  * Table constraints
+//!
+//! A database catalog, schema, and table are represented by a type implementing
+//! [DatabaseCatalogEntry], [DatabaseSchemaEntry], and [DatabaseTableEntry],
+//! respectively. These can be concrete Rust structs, such as [SimpleCatalogEntry],
+//! [SimpleSchemaEntry], and [SimpleTableEntry]. Or they can be zero-copy views
+//! onto Arrow record batches as returned by the C API ADBC drivers (TODO).
+//!
+//! | Trait                        | Simple Rust-based    |
+//! |------------------------------|----------------------|
+//! | [DatabaseCatalogCollection]  | [SimpleSchemaEntry]  |
+//! | [DatabaseCatalogEntry]       | [SimpleCatalogEntry] |
+//! | [DatabaseSchemaEntry]        | [SimpleSchemaEntry]  |
+//! | [DatabaseTableEntry]         | [SimpleTableEntry]   |
+//!
+//! There are owned and reference variations of columns, table constraints,
+//! and foreign key usage. Each have a `borrow()` method to transform a owned
+//! variant into its reference variant, and a `to_owned()` method to transform the
+//! reference variant into the owned. These mimic the [std::borrow::Borrow] and
+//! [std::borrow::ToOwned] traits, but do not actually implement them.
+//!
+//! | Owned             | Reference            |
+//! |-------------------|----------------------|
+//! | [ColumnSchema]    | [ColumnSchemaRef]    |
+//! | [TableConstraint] | [TableConstraintRef] |
+//! | [ForeignKeyUsage] | [ForeignKeyUsageRef] |
+
+/// A collection of database catalogs, returned by [crate::AdbcConnection::get_objects].
+pub trait DatabaseCatalogCollection {
+    type CatalogEntryType<'a>: DatabaseCatalogEntry<'a>
+    where
+        Self: 'a;
+
+    /// List all catalogs in the result set.
+    fn catalogs<'a>(&'a self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'a>> + 'a>;

Review Comment:
   You can use lifetime elision with `'_`:
   ```suggestion
       fn catalogs(&self) -> Box<dyn Iterator<Item = Self::CatalogEntryType<'_>> + '_>;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: github-unsubscribe@arrow.apache.org

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